Revert source incompatible sharding changes (#24126)
* Revert "fix entityPropsFactory id param, #21809" This reverts commitcd7eae28f6. * Revert "Merge pull request #24058 from talpr/talpr-24053-add-entity-id-to-sharding-props" This reverts commit8417e70460, reversing changes made to22e85f869d.
This commit is contained in:
parent
cd7eae28f6
commit
582f6a4836
7 changed files with 81 additions and 276 deletions
|
|
@ -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,
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue