Revert source incompatible sharding changes (#24126)

* Revert "fix entityPropsFactory id param, #21809"
This reverts commit cd7eae28f6.
* Revert "Merge pull request #24058 from talpr/talpr-24053-add-entity-id-to-sharding-props"
This reverts commit 8417e70460, reversing
changes made to 22e85f869d.
This commit is contained in:
Johan Andrén 2017-12-07 17:49:29 +01:00 committed by GitHub
parent cd7eae28f6
commit 582f6a4836
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
7 changed files with 81 additions and 276 deletions

View file

@ -3,10 +3,11 @@
*/
package akka.cluster.sharding
import akka.cluster.ddata.{ Replicator, ReplicatorSettings }
import akka.cluster.sharding.ShardCoordinator.Internal.{ HandOff, ShardStopped }
import akka.cluster.sharding.ShardRegion.{ CurrentRegions, GetCurrentRegions, Passivate }
import akka.cluster.ddata.{ ReplicatorSettings, Replicator }
import akka.cluster.sharding.ShardCoordinator.Internal.{ ShardStopped, HandOff }
import akka.cluster.sharding.ShardRegion.Passivate
import akka.cluster.sharding.ShardRegion.GetCurrentRegions
import akka.cluster.sharding.ShardRegion.CurrentRegions
import language.postfixOps
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
@ -23,7 +24,6 @@ import akka.remote.testkit.STMultiNodeSpec
import akka.testkit._
import akka.testkit.TestEvent.Mute
import java.io.File
import org.apache.commons.io.FileUtils
import akka.cluster.singleton.ClusterSingletonManager
import akka.cluster.singleton.ClusterSingletonManagerSettings
@ -39,17 +39,13 @@ object ClusterShardingSpec {
case object Stop
final case class CounterChanged(delta: Int)
object Counter {
val ShardingTypeName: String = "Counter"
def props(id: String): Props = Props(new Counter(id))
}
class Counter(id: String) extends PersistentActor {
class Counter extends PersistentActor {
import ShardRegion.Passivate
context.setReceiveTimeout(120.seconds)
override def persistenceId: String = s"${Counter.ShardingTypeName}-$id"
// self.path.name is the entity identifier (utf-8 URL-encoded)
override def persistenceId: String = "Counter-" + self.path.name
var count = 0
//#counter-actor
@ -91,29 +87,18 @@ object ClusterShardingSpec {
case ShardRegion.StartEntity(id) (id.toLong % numberOfShards).toString
}
object QualifiedCounter {
def props(typeName: String, id: String): Props = Props(new QualifiedCounter(typeName, id))
def qualifiedCounterProps(typeName: String): Props =
Props(new QualifiedCounter(typeName))
class QualifiedCounter(typeName: String) extends Counter {
override def persistenceId: String = typeName + "-" + self.path.name
}
class QualifiedCounter(typeName: String, id: String) extends Counter(id) {
override def persistenceId: String = s"$typeName-$id"
}
object AnotherCounter {
val ShardingTypeName: String = "AnotherCounter"
def props(id: String): Props = Props(new AnotherCounter(id))
}
class AnotherCounter(id: String) extends QualifiedCounter(AnotherCounter.ShardingTypeName, id)
class AnotherCounter extends QualifiedCounter("AnotherCounter")
//#supervisor
object CounterSupervisor {
val ShardingTypeName: String = "CounterSupervisor"
def props(id: String): Props = Props(new CounterSupervisor(id))
}
class CounterSupervisor(entityId: String) extends Actor {
val counter = context.actorOf(Counter.props(entityId), "theCounter")
class CounterSupervisor extends Actor {
val counter = context.actorOf(Props[Counter], "theCounter")
override val supervisorStrategy = OneForOneStrategy() {
case _: IllegalArgumentException SupervisorStrategy.Resume
@ -128,9 +113,6 @@ object ClusterShardingSpec {
}
//#supervisor
// must use different unique name for some tests than the one used in API tests
val TestCounterShardingTypeName = s"Test${Counter.ShardingTypeName}"
}
abstract class ClusterShardingSpecConfig(
@ -316,7 +298,7 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
ShardCoordinator.props(typeName, settings, allocationStrategy, replicator, majorityMinCap)
}
List(TestCounterShardingTypeName, "rebalancingCounter", "RememberCounterEntities", "AnotherRememberCounter",
List("counter", "rebalancingCounter", "RememberCounterEntities", "AnotherRememberCounter",
"RememberCounter", "RebalancingRememberCounter", "AutoMigrateRememberRegionTest").foreach { typeName
val rebalanceEnabled = typeName.toLowerCase.startsWith("rebalancing")
val rememberEnabled = typeName.toLowerCase.contains("remember")
@ -347,7 +329,7 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
system.actorOf(
ShardRegion.props(
typeName = typeName,
entityPropsFactory = entityId QualifiedCounter.props(typeName, entityId),
entityProps = qualifiedCounterProps(typeName),
settings = settings,
coordinatorPath = "/user/" + typeName + "Coordinator/singleton/coordinator",
extractEntityId = extractEntityId,
@ -358,7 +340,7 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
name = typeName + "Region")
}
lazy val region = createRegion(TestCounterShardingTypeName, rememberEntities = false)
lazy val region = createRegion("counter", rememberEntities = false)
lazy val rebalancingRegion = createRegion("rebalancingCounter", rememberEntities = false)
lazy val persistentEntitiesRegion = createRegion("RememberCounterEntities", rememberEntities = true)
@ -430,7 +412,7 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
region ! EntityEnvelope(2, Increment)
region ! Get(2)
expectMsg(3)
lastSender.path should ===(node(second) / "user" / s"${TestCounterShardingTypeName}Region" / "2" / "2")
lastSender.path should ===(node(second) / "user" / "counterRegion" / "2" / "2")
region ! Get(11)
expectMsg(1)
@ -438,7 +420,7 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
lastSender.path should ===(region.path / "11" / "11")
region ! Get(12)
expectMsg(1)
lastSender.path should ===(node(second) / "user" / s"${TestCounterShardingTypeName}Region" / "0" / "12")
lastSender.path should ===(node(second) / "user" / "counterRegion" / "0" / "12")
}
enterBarrier("first-update")
@ -477,10 +459,10 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
val settings = ClusterShardingSettings(cfg)
val proxy = system.actorOf(
ShardRegion.proxyProps(
typeName = TestCounterShardingTypeName,
typeName = "counter",
dataCenter = None,
settings,
coordinatorPath = s"/user/${TestCounterShardingTypeName}Coordinator/singleton/coordinator",
coordinatorPath = "/user/counterCoordinator/singleton/coordinator",
extractEntityId = extractEntityId,
extractShardId = extractShardId,
system.deadLetters,
@ -555,12 +537,12 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
region ! EntityEnvelope(3, Increment)
region ! Get(3)
expectMsg(11)
lastSender.path should ===(node(third) / "user" / s"${TestCounterShardingTypeName}Region" / "3" / "3")
lastSender.path should ===(node(third) / "user" / "counterRegion" / "3" / "3")
region ! EntityEnvelope(4, Increment)
region ! Get(4)
expectMsg(21)
lastSender.path should ===(node(fourth) / "user" / s"${TestCounterShardingTypeName}Region" / "4" / "4")
lastSender.path should ===(node(fourth) / "user" / "counterRegion" / "4" / "4")
}
enterBarrier("first-update")
@ -593,7 +575,7 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
within(1.second) {
region.tell(Get(3), probe3.ref)
probe3.expectMsg(11)
probe3.lastSender.path should ===(node(third) / "user" / s"${TestCounterShardingTypeName}Region" / "3" / "3")
probe3.lastSender.path should ===(node(third) / "user" / "counterRegion" / "3" / "3")
}
}
val probe4 = TestProbe()
@ -601,7 +583,7 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
within(1.second) {
region.tell(Get(4), probe4.ref)
probe4.expectMsg(21)
probe4.lastSender.path should ===(node(fourth) / "user" / s"${TestCounterShardingTypeName}Region" / "4" / "4")
probe4.lastSender.path should ===(node(fourth) / "user" / "counterRegion" / "4" / "4")
}
}
@ -646,23 +628,23 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
runOn(third, fourth, fifth, sixth) {
//#counter-start
val counterRegion: ActorRef = ClusterSharding(system).start(
typeName = Counter.ShardingTypeName,
entityPropsFactory = entityId Counter.props(entityId),
typeName = "Counter",
entityProps = Props[Counter],
settings = ClusterShardingSettings(system),
extractEntityId = extractEntityId,
extractShardId = extractShardId)
//#counter-start
ClusterSharding(system).start(
typeName = AnotherCounter.ShardingTypeName,
entityPropsFactory = entityId AnotherCounter.props(entityId),
typeName = "AnotherCounter",
entityProps = Props[AnotherCounter],
settings = ClusterShardingSettings(system),
extractEntityId = extractEntityId,
extractShardId = extractShardId)
//#counter-supervisor-start
ClusterSharding(system).start(
typeName = CounterSupervisor.ShardingTypeName,
entityPropsFactory = entityId CounterSupervisor.props(entityId),
typeName = "SupervisedCounter",
entityProps = Props[CounterSupervisor],
settings = ClusterShardingSettings(system),
extractEntityId = extractEntityId,
extractShardId = extractShardId)
@ -671,18 +653,17 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
enterBarrier("extension-started")
runOn(fifth) {
//#counter-usage
val counterRegion: ActorRef = ClusterSharding(system).shardRegion(Counter.ShardingTypeName)
val entityId = 999
counterRegion ! Get(entityId)
val counterRegion: ActorRef = ClusterSharding(system).shardRegion("Counter")
counterRegion ! Get(123)
expectMsg(0)
counterRegion ! EntityEnvelope(entityId, Increment)
counterRegion ! Get(entityId)
counterRegion ! EntityEnvelope(123, Increment)
counterRegion ! Get(123)
expectMsg(1)
//#counter-usage
ClusterSharding(system).shardRegion(AnotherCounter.ShardingTypeName) ! EntityEnvelope(entityId, Decrement)
ClusterSharding(system).shardRegion(AnotherCounter.ShardingTypeName) ! Get(entityId)
ClusterSharding(system).shardRegion("AnotherCounter") ! EntityEnvelope(123, Decrement)
ClusterSharding(system).shardRegion("AnotherCounter") ! Get(123)
expectMsg(-1)
}
@ -691,8 +672,8 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
// sixth is a frontend node, i.e. proxy only
runOn(sixth) {
for (n 1000 to 1010) {
ClusterSharding(system).shardRegion(Counter.ShardingTypeName) ! EntityEnvelope(n, Increment)
ClusterSharding(system).shardRegion(Counter.ShardingTypeName) ! Get(n)
ClusterSharding(system).shardRegion("Counter") ! EntityEnvelope(n, Increment)
ClusterSharding(system).shardRegion("Counter") ! Get(n)
expectMsg(1)
lastSender.path.address should not be (Cluster(system).selfAddress)
}
@ -705,7 +686,7 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
runOn(first) {
val counterRegionViaStart: ActorRef = ClusterSharding(system).start(
typeName = "ApiTest",
entityPropsFactory = Counter.props,
entityProps = Props[Counter],
settings = ClusterShardingSettings(system),
extractEntityId = extractEntityId,
extractShardId = extractShardId)
@ -722,7 +703,7 @@ abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends Mu
runOn(sixth) {
// #proxy-dc
val counterProxyDcB: ActorRef = ClusterSharding(system).startProxy(
typeName = Counter.ShardingTypeName,
typeName = "Counter",
role = None,
dataCenter = Some("B"),
extractEntityId = extractEntityId,