You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@pekko.apache.org by "Roiocam (via GitHub)" <gi...@apache.org> on 2024/03/20 09:01:32 UTC

[PR] feat: durable state database support [incubator-pekko-persistence-jdbc]

Roiocam opened a new pull request, #158:
URL: https://github.com/apache/incubator-pekko-persistence-jdbc/pull/158

   # Motivation
   
   resolves: #56 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "He-Pin (via GitHub)" <gi...@apache.org>.
He-Pin commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1557539305


##########
core/src/main/scala/org/apache/pekko/persistence/jdbc/state/SequenceNextValUpdater.scala:
##########
@@ -60,3 +60,31 @@ import slick.sql.SqlStreamingAction
 
   def getSequenceNextValueExpr() = sql"""#$nextValFetcher""".as[String]
 }
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[jdbc] class SqlServerSequenceNextValUpdater(profile: JdbcProfile,
+    val durableStateTableCfg: DurableStateTableConfiguration)
+    extends SequenceNextValUpdater {
+
+  import profile.api._
+
+  final val nextValFetcher = s"""SELECT NEXT VALUE FOR ${durableStateTableCfg.columnNames.globalOffset}""".stripMargin

Review Comment:
   I think that maybe not safe with string template for sql



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [incubator-pekko-persistence-jdbc]

Posted by "mdedetrich (via GitHub)" <gi...@apache.org>.
mdedetrich commented on PR #158:
URL: https://github.com/apache/incubator-pekko-persistence-jdbc/pull/158#issuecomment-2009337712

   > Would it be possible to add string constants for all these Strings?
   > 
   > ```
   >       case PostgresProfile  => "Postgres"
   >       case MySQLProfile     => "MySQL"
   >       case OracleProfile    => "Oracle"
   >       case SQLServerProfile => "SqlServer"
   >       case H2Profile        => "H2"
   > ```
   > 
   > And then to remove the direct usage of the strings - use the constants instead.
   
   Definite +1 from me


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "Roiocam (via GitHub)" <gi...@apache.org>.
Roiocam commented on PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#issuecomment-2042258489

   > Unfortunately, the Oracle integration tests are failing due to table name issues.
   
   Seems like latest commit break this, I will figure out later 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "He-Pin (via GitHub)" <gi...@apache.org>.
He-Pin commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1550936475


##########
core/src/main/scala/org/apache/pekko/persistence/jdbc/state/DurableStateQueries.scala:
##########
@@ -31,22 +26,17 @@ import pekko.persistence.jdbc.config.DurableStateTableConfiguration
     val profile: JdbcProfile,
     override val durableStateTableCfg: DurableStateTableConfiguration)
     extends DurableStateTables {
+
   import profile.api._
 
-  private def slickProfileToSchemaType(profile: JdbcProfile): String =
-    profile match {
-      case PostgresProfile  => "Postgres"
-      case MySQLProfile     => "MySQL"
-      case OracleProfile    => "Oracle"
-      case SQLServerProfile => "SqlServer"
-      case H2Profile        => "H2"
-      case _                => throw new IllegalArgumentException(s"Unknown JdbcProfile $profile encountered")
-    }
-
-  lazy val sequenceNextValUpdater = slickProfileToSchemaType(profile) match {
-    case "H2"       => new H2SequenceNextValUpdater(profile, durableStateTableCfg)
-    case "Postgres" => new PostgresSequenceNextValUpdater(profile, durableStateTableCfg)
-    case _          => ???
+  lazy val sequenceNextValUpdater = profile match {
+    case H2Profile        => new H2SequenceNextValUpdater(profile, durableStateTableCfg)
+    case PostgresProfile  => new PostgresSequenceNextValUpdater(profile, durableStateTableCfg)
+    case SQLServerProfile => new SqlServerSequenceNextValUpdater(profile, durableStateTableCfg)
+    case OracleProfile    => new OracleSequenceNextValUpdater(profile, durableStateTableCfg)
+    // TODO https://github.com/apache/pekko-persistence-jdbc/pull/158
+    // case MySQLProfile     => new MySQLSequenceNextValUpdater(profile, durableStateTableCfg)
+    case _ => throw new UnsupportedOperationException(s"Currently JdbcProfile: <$profile> wasn't support.")

Review Comment:
   Unsupported JdbcProfile <$profile> for durableState.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "Roiocam (via GitHub)" <gi...@apache.org>.
Roiocam commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1555146278


##########
core/src/main/resources/schema/oracle/oracle-create-schema.sql:
##########
@@ -55,3 +55,34 @@ BEGIN
   EXECUTE IMMEDIATE 'ALTER SEQUENCE EVENT_JOURNAL__ORDERING_SEQ INCREMENT BY 1 MINVALUE 0';
 END;
 /
+
+CREATE SEQUENCE DURABLE_STATE__GLOBAL_OFFSET_SEQ START WITH 1 INCREMENT BY 1 NOMAXVALUE
+/
+
+CREATE TABLE DURABLE_STATE
+(
+    GLOBAL_OFFSET         NUMERIC      NOT NULL,
+    PERSISTENCE_ID        VARCHAR(255) NOT NULL,
+    REVISION              NUMERIC      NOT NULL,
+    STATE_PAYLOAD         BLOB         NOT NULL,
+    STATE_SERIAL_ID       NUMBER(10) NOT NULL,

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "laglangyue (via GitHub)" <gi...@apache.org>.
laglangyue commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1550928951


##########
core/src/main/resources/schema/sqlserver/sqlserver-create-schema.sql:
##########
@@ -40,3 +40,24 @@ CREATE TABLE "snapshot" (
     PRIMARY KEY ("persistence_id", "sequence_number")
   )
 
+-- Create Sequence Object
+CREATE SEQUENCE global_offset
+    START WITH 1
+    INCREMENT BY 1;
+
+CREATE TABLE durable_state
+(
+    "global_offset"         BIGINT
+        CONSTRAINT [df_global_offset] DEFAULT
+        (NEXT VALUE FOR global_offset),
+    "persistence_id"        VARCHAR(255)   NOT NULL,
+    "revision"              NUMERIC(10, 0) NOT NULL,
+    "state_payload"         VARBINARY(MAX) NOT NULL,
+    "state_serial_id"       INTEGER        NOT NULL,
+    "state_serial_manifest" VARCHAR(MAX),

Review Comment:
   can we use text, varchar(max) is strange



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "Roiocam (via GitHub)" <gi...@apache.org>.
Roiocam commented on PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#issuecomment-2036146336

   > There's a fix for the MySQL support over in the akka repo:
   > 
   > [akka/akka-persistence-jdbc#803](https://github.com/akka/akka-persistence-jdbc/pull/803)
   > 
   > What's the procedure for mirroring these kinds of fixes in the pekko repo? Are there some licensing issues? Is it just a matter of copying the change set and submitting a PR? Is there some automated process?
   
   Yes, there are license issues, but it is not the most important thing for this PR.
   
   I hadn't seen any code from akka PR before, and then take a short look after you mentioned it. If you review it carefully, you will notice that I already implemented MySQL support on my early commits, but I dropped those after some investigation.
   
   Because currently durable state `global_offset` implementation is based on sequence and Pekko Persistence JDBC will increase sequence after an update or insert. In MySQL, seems likes you can't increase sequence manually, that's why you will see the comment from the Akka maintainer that MySQLSequenceNextValueExpr doesn't work.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "He-Pin (via GitHub)" <gi...@apache.org>.
He-Pin commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1550937037


##########
integration-test/src/test/scala/org/apache/pekko/persistence/jdbc/integration/ScalaJdbcDurableStateStoreQueryTest.scala:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.pekko.persistence.jdbc.integration
+
+import com.typesafe.config.ConfigFactory
+import org.apache.pekko
+import pekko.actor.ActorSystem
+import pekko.persistence.jdbc.state.scaladsl.JdbcDurableStateSpec
+import pekko.persistence.jdbc.testkit.internal.{ Oracle, Postgres, SchemaType, SqlServer }
+
+class PostgresScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("postgres-shared-db-application.conf"), Postgres) {
+  implicit lazy val system: ActorSystem =
+    ActorSystem("JdbcDurableStateSpec", config.withFallback(customSerializers))

Review Comment:
   as These lines are identical as the old, so we may need keep the lightbend header.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "pjfanning (via GitHub)" <gi...@apache.org>.
pjfanning commented on PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#issuecomment-2042221420

   Unfortunately, the Oracle integration tests are failing due to table name issues.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [incubator-pekko-persistence-jdbc]

Posted by "pjfanning (via GitHub)" <gi...@apache.org>.
pjfanning commented on code in PR #158:
URL: https://github.com/apache/incubator-pekko-persistence-jdbc/pull/158#discussion_r1531743656


##########
core/src/main/scala/org/apache/pekko/persistence/jdbc/state/DurableStateQueries.scala:
##########
@@ -46,6 +46,7 @@ import pekko.persistence.jdbc.config.DurableStateTableConfiguration
   lazy val sequenceNextValUpdater = slickProfileToSchemaType(profile) match {
     case "H2"       => new H2SequenceNextValUpdater(profile, durableStateTableCfg)
     case "Postgres" => new PostgresSequenceNextValUpdater(profile, durableStateTableCfg)
+    case "MySQL"    => new MySQLSequenceNextValUpdater(profile, durableStateTableCfg)
     case _          => ???

Review Comment:
   unfortunately - this support is not standardised across all db servers - we will need specific versions for different db types



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "He-Pin (via GitHub)" <gi...@apache.org>.
He-Pin commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1550944856


##########
core/src/main/scala/org/apache/pekko/persistence/jdbc/state/SequenceNextValUpdater.scala:
##########
@@ -60,3 +60,31 @@ import slick.sql.SqlStreamingAction
 
   def getSequenceNextValueExpr() = sql"""#$nextValFetcher""".as[String]
 }
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[jdbc] class SqlServerSequenceNextValUpdater(profile: JdbcProfile,
+    val durableStateTableCfg: DurableStateTableConfiguration)
+    extends SequenceNextValUpdater {
+
+  import profile.api._
+
+  final val nextValFetcher = s"""SELECT NEXT VALUE FOR ${durableStateTableCfg.columnNames.globalOffset}""".stripMargin
+
+  def getSequenceNextValueExpr() = sql"""#$nextValFetcher""".as[String]
+}
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[jdbc] class OracleSequenceNextValUpdater(profile: JdbcProfile,
+    val durableStateTableCfg: DurableStateTableConfiguration)

Review Comment:
   `val` is not needed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "Roiocam (via GitHub)" <gi...@apache.org>.
Roiocam commented on PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#issuecomment-2033750470

   It may need another separate PR for MySQL support.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "Roiocam (via GitHub)" <gi...@apache.org>.
Roiocam commented on PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#issuecomment-2045502509

   All suggestions have been apply, there is changes: https://github.com/apache/pekko-persistence-jdbc/pull/158/files/1d3b1e5bf22ac6f7a3cf8c68fd0ad883f3d7bd84..c7bc649641a01c9471754852398d4f44299d9023


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [incubator-pekko-persistence-jdbc]

Posted by "laglangyue (via GitHub)" <gi...@apache.org>.
laglangyue commented on PR #158:
URL: https://github.com/apache/incubator-pekko-persistence-jdbc/pull/158#issuecomment-2009088807

   how to test it,is there any integration testing?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "anchann (via GitHub)" <gi...@apache.org>.
anchann commented on PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#issuecomment-2036178832

   Yeah, I see the comment and the revert now, thank you for pointing out. Sorry, should have been more careful before posting.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "pjfanning (via GitHub)" <gi...@apache.org>.
pjfanning commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1557529665


##########
integration-test/src/test/scala/org/apache/pekko/persistence/jdbc/integration/ScalaJdbcDurableStateStoreQueryTest.scala:
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.pekko.persistence.jdbc.integration
+
+import com.typesafe.config.ConfigFactory
+import org.apache.pekko
+import pekko.actor.ActorSystem
+import pekko.persistence.jdbc.state.scaladsl.JdbcDurableStateSpec
+import pekko.persistence.jdbc.testkit.internal.{ Oracle, Postgres, SchemaType, SqlServer }
+
+class OracleScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("oracle-shared-db-application.conf"), Oracle) {
+  implicit lazy val system: ActorSystem =
+    ActorSystem("JdbcDurableStateSpec", config.withFallback(customSerializers))
+
+  override private[jdbc] def dropAndCreate(schemaType: SchemaType): Unit = {
+    super.dropAndCreate(schemaType)
+    withStatement(stmt => stmt.executeUpdate("""BEGIN "reset__global_offset"; END; """))
+  }
+}
+
+class SqlServerScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("sqlserver-shared-db-application.conf"), SqlServer) {
+  implicit lazy val system: ActorSystem =
+    ActorSystem("JdbcDurableStateSpec", config.withFallback(customSerializers))
+
+  override private[jdbc] def dropAndCreate(schemaType: SchemaType): Unit = {
+    super.dropAndCreate(schemaType)
+    withStatement(stmt => stmt.executeUpdate("""ALTER SEQUENCE global_offset RESTART WITH 1""".stripMargin))

Review Comment:
   the stripMargin call is not needed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "Roiocam (via GitHub)" <gi...@apache.org>.
Roiocam commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1551940211


##########
core/src/main/scala/org/apache/pekko/persistence/jdbc/state/SequenceNextValUpdater.scala:
##########
@@ -60,3 +60,31 @@ import slick.sql.SqlStreamingAction
 
   def getSequenceNextValueExpr() = sql"""#$nextValFetcher""".as[String]
 }
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[jdbc] class SqlServerSequenceNextValUpdater(profile: JdbcProfile,
+    val durableStateTableCfg: DurableStateTableConfiguration)
+    extends SequenceNextValUpdater {
+
+  import profile.api._
+
+  final val nextValFetcher = s"""SELECT NEXT VALUE FOR ${durableStateTableCfg.columnNames.globalOffset}""".stripMargin
+
+  def getSequenceNextValueExpr() = sql"""#$nextValFetcher""".as[String]
+}
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[jdbc] class OracleSequenceNextValUpdater(profile: JdbcProfile,
+    val durableStateTableCfg: DurableStateTableConfiguration)

Review Comment:
   Let us keep this `val`, otherwise it will complain unused warning, maybe we will use this cfg for the SQL template in the future to avoid binary incompatibility.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "He-Pin (via GitHub)" <gi...@apache.org>.
He-Pin commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1550941295


##########
core/src/main/resources/schema/sqlserver/sqlserver-drop-schema.sql:
##########
@@ -1,3 +1,4 @@
 DROP TABLE IF EXISTS event_tag;
 DROP TABLE IF EXISTS event_journal;
 DROP TABLE IF EXISTS snapshot;
+DROP TABLE IF EXISTS durable_state;

Review Comment:
   global_offset?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "pjfanning (via GitHub)" <gi...@apache.org>.
pjfanning commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1557528120


##########
core/src/main/scala/org/apache/pekko/persistence/jdbc/state/SequenceNextValUpdater.scala:
##########
@@ -60,3 +60,31 @@ import slick.sql.SqlStreamingAction
 
   def getSequenceNextValueExpr() = sql"""#$nextValFetcher""".as[String]
 }
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[jdbc] class SqlServerSequenceNextValUpdater(profile: JdbcProfile,
+    val durableStateTableCfg: DurableStateTableConfiguration)
+    extends SequenceNextValUpdater {
+
+  import profile.api._
+
+  final val nextValFetcher = s"""SELECT NEXT VALUE FOR ${durableStateTableCfg.columnNames.globalOffset}""".stripMargin

Review Comment:
   this doesn't need stripMargin call



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "He-Pin (via GitHub)" <gi...@apache.org>.
He-Pin commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1557539807


##########
core/src/test/scala/org/apache/pekko/persistence/jdbc/state/scaladsl/JdbcDurableStateSpec.scala:
##########
@@ -83,7 +83,14 @@ abstract class JdbcDurableStateSpec(config: Config, schemaType: SchemaType) exte
             e shouldBe an[org.h2.jdbc.JdbcSQLIntegrityConstraintViolationException]
           case Postgres =>
             e shouldBe an[org.postgresql.util.PSQLException]
-          case _ => ???
+          // TODO https://github.com/apache/pekko-persistence-jdbc/pull/158

Review Comment:
   Open up an issue about the mysql is not supported yet?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "pjfanning (via GitHub)" <gi...@apache.org>.
pjfanning commented on PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#issuecomment-2043418539

   @Roiocam can you create an issue for following up with MySQL support - so at least, the issue is there for other users to examine and maybe provide a solution?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [incubator-pekko-persistence-jdbc]

Posted by "Roiocam (via GitHub)" <gi...@apache.org>.
Roiocam commented on code in PR #158:
URL: https://github.com/apache/incubator-pekko-persistence-jdbc/pull/158#discussion_r1531751190


##########
core/src/main/scala/org/apache/pekko/persistence/jdbc/state/DurableStateQueries.scala:
##########
@@ -46,6 +46,7 @@ import pekko.persistence.jdbc.config.DurableStateTableConfiguration
   lazy val sequenceNextValUpdater = slickProfileToSchemaType(profile) match {
     case "H2"       => new H2SequenceNextValUpdater(profile, durableStateTableCfg)
     case "Postgres" => new PostgresSequenceNextValUpdater(profile, durableStateTableCfg)
+    case "MySQL"    => new MySQLSequenceNextValUpdater(profile, durableStateTableCfg)
     case _          => ???

Review Comment:
   > `case _ => ???` Can we provide a default implementation,it is another feature
   
   Basically, this is a database dialect. As pfj said, this feature is currently no standardized on different databases.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "pjfanning (via GitHub)" <gi...@apache.org>.
pjfanning commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1552426195


##########
core/src/main/resources/schema/oracle/oracle-create-schema.sql:
##########
@@ -55,3 +55,34 @@ BEGIN
   EXECUTE IMMEDIATE 'ALTER SEQUENCE EVENT_JOURNAL__ORDERING_SEQ INCREMENT BY 1 MINVALUE 0';
 END;
 /
+
+CREATE SEQUENCE DURABLE_STATE__GLOBAL_OFFSET_SEQ START WITH 1 INCREMENT BY 1 NOMAXVALUE
+/
+
+CREATE TABLE DURABLE_STATE
+(
+    GLOBAL_OFFSET         NUMERIC      NOT NULL,
+    PERSISTENCE_ID        VARCHAR(255) NOT NULL,
+    REVISION              NUMERIC      NOT NULL,
+    STATE_PAYLOAD         BLOB         NOT NULL,
+    STATE_SERIAL_ID       NUMBER(10) NOT NULL,

Review Comment:
   minor format issue - can you add a space before 'NOT'?



##########
core/src/main/resources/schema/oracle/oracle-create-schema.sql:
##########
@@ -55,3 +55,34 @@ BEGIN
   EXECUTE IMMEDIATE 'ALTER SEQUENCE EVENT_JOURNAL__ORDERING_SEQ INCREMENT BY 1 MINVALUE 0';
 END;
 /
+
+CREATE SEQUENCE DURABLE_STATE__GLOBAL_OFFSET_SEQ START WITH 1 INCREMENT BY 1 NOMAXVALUE
+/
+
+CREATE TABLE DURABLE_STATE
+(
+    GLOBAL_OFFSET         NUMERIC      NOT NULL,
+    PERSISTENCE_ID        VARCHAR(255) NOT NULL,
+    REVISION              NUMERIC      NOT NULL,
+    STATE_PAYLOAD         BLOB         NOT NULL,
+    STATE_SERIAL_ID       NUMBER(10) NOT NULL,
+    STATE_SERIAL_MANIFEST VARCHAR(255),
+    TAG                   VARCHAR(255) NOT NULL,
+    STATE_TIMESTAMP       NUMERIC      NOT NULL,
+    PRIMARY KEY (PERSISTENCE_ID)
+)
+/
+
+CREATE OR REPLACE TRIGGER DURABLE_STATE__GLOBAL_OFFSET_SEQ_TRG before insert on DURABLE_STATE REFERENCING NEW AS NEW FOR EACH ROW WHEN (new.GLOBAL_OFFSET is null) begin select DURABLE_STATE__GLOBAL_OFFSET_SEQ.nextval into :new.GLOBAL_OFFSET from sys.dual; end;

Review Comment:
   any idea why this can't be spread over multiple lines while other DDL commands are?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "He-Pin (via GitHub)" <gi...@apache.org>.
He-Pin commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1550937315


##########
integration-test/src/test/scala/org/apache/pekko/persistence/jdbc/integration/ScalaJdbcDurableStateStoreQueryTest.scala:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.pekko.persistence.jdbc.integration
+
+import com.typesafe.config.ConfigFactory
+import org.apache.pekko
+import pekko.actor.ActorSystem
+import pekko.persistence.jdbc.state.scaladsl.JdbcDurableStateSpec
+import pekko.persistence.jdbc.testkit.internal.{ Oracle, Postgres, SchemaType, SqlServer }
+
+class PostgresScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("postgres-shared-db-application.conf"), Postgres) {
+  implicit lazy val system: ActorSystem =
+    ActorSystem("JdbcDurableStateSpec", config.withFallback(customSerializers))
+}
+
+class OracleScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("oracle-shared-db-application.conf"), Oracle) {
+  implicit lazy val system: ActorSystem =
+    ActorSystem("JdbcDurableStateSpec", config.withFallback(customSerializers))
+
+  override private[jdbc] def dropAndCreate(schemaType: SchemaType): Unit = {
+    super.dropAndCreate(schemaType)
+    withStatement(stmt => stmt.executeUpdate("""BEGIN "reset__global_offset"; END; """))
+  }
+}
+
+class SqlServerScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("sqlserver-shared-db-application.conf"), SqlServer) {
+  implicit lazy val system: ActorSystem =
+    ActorSystem("JdbcDurableStateSpec", config.withFallback(customSerializers))
+
+  override private[jdbc] def dropAndCreate(schemaType: SchemaType): Unit = {
+    super.dropAndCreate(schemaType)
+    withStatement(stmt => stmt.executeUpdate("""ALTER SEQUENCE global_offset RESTART WITH 1""".stripMargin))
+  }

Review Comment:
   You can swith these tests to a dedicated file with apache header.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "laglangyue (via GitHub)" <gi...@apache.org>.
laglangyue commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1550927137


##########
core/src/main/resources/schema/sqlserver/sqlserver-create-schema.sql:
##########
@@ -40,3 +40,24 @@ CREATE TABLE "snapshot" (
     PRIMARY KEY ("persistence_id", "sequence_number")
   )
 
+-- Create Sequence Object
+CREATE SEQUENCE global_offset
+    START WITH 1
+    INCREMENT BY 1;
+
+CREATE TABLE durable_state
+(
+    "global_offset"         BIGINT
+        CONSTRAINT [df_global_offset] DEFAULT
+        (NEXT VALUE FOR global_offset),
+    "persistence_id"        VARCHAR(255)   NOT NULL,
+    "revision"              NUMERIC(10, 0) NOT NULL,

Review Comment:
   why not use bigint, because pg use bigint



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "He-Pin (via GitHub)" <gi...@apache.org>.
He-Pin commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1557541538


##########
core/src/main/scala/org/apache/pekko/persistence/jdbc/state/SequenceNextValUpdater.scala:
##########
@@ -60,3 +60,31 @@ import slick.sql.SqlStreamingAction
 
   def getSequenceNextValueExpr() = sql"""#$nextValFetcher""".as[String]
 }
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[jdbc] class SqlServerSequenceNextValUpdater(profile: JdbcProfile,

Review Comment:
   final class



##########
core/src/main/scala/org/apache/pekko/persistence/jdbc/state/SequenceNextValUpdater.scala:
##########
@@ -60,3 +60,31 @@ import slick.sql.SqlStreamingAction
 
   def getSequenceNextValueExpr() = sql"""#$nextValFetcher""".as[String]
 }
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[jdbc] class SqlServerSequenceNextValUpdater(profile: JdbcProfile,
+    val durableStateTableCfg: DurableStateTableConfiguration)
+    extends SequenceNextValUpdater {
+
+  import profile.api._
+
+  final val nextValFetcher = s"""SELECT NEXT VALUE FOR ${durableStateTableCfg.columnNames.globalOffset}""".stripMargin
+
+  def getSequenceNextValueExpr() = sql"""#$nextValFetcher""".as[String]
+}
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[jdbc] class OracleSequenceNextValUpdater(profile: JdbcProfile,

Review Comment:
   final class



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "pjfanning (via GitHub)" <gi...@apache.org>.
pjfanning commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1557603521


##########
core/src/test/scala/org/apache/pekko/persistence/jdbc/state/scaladsl/JdbcDurableStateSpec.scala:
##########
@@ -83,7 +83,14 @@ abstract class JdbcDurableStateSpec(config: Config, schemaType: SchemaType) exte
             e shouldBe an[org.h2.jdbc.JdbcSQLIntegrityConstraintViolationException]
           case Postgres =>
             e shouldBe an[org.postgresql.util.PSQLException]
-          case _ => ???
+          // TODO https://github.com/apache/pekko-persistence-jdbc/pull/158

Review Comment:
   The issue exists and there is a code comment with a link. #174 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [incubator-pekko-persistence-jdbc]

Posted by "pjfanning (via GitHub)" <gi...@apache.org>.
pjfanning commented on PR #158:
URL: https://github.com/apache/incubator-pekko-persistence-jdbc/pull/158#issuecomment-2009330366

   Would it be possible to add string constants for all these Strings?
   
   ```
         case PostgresProfile  => "Postgres"
         case MySQLProfile     => "MySQL"
         case OracleProfile    => "Oracle"
         case SQLServerProfile => "SqlServer"
         case H2Profile        => "H2"
   ```
   
   And then to remove the direct usage of the strings - use the constants instead.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "laglangyue (via GitHub)" <gi...@apache.org>.
laglangyue commented on PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#issuecomment-2036206449

   LGTM.
   I am not very familiar with persistence. I am just comparing it with the implementation of PG


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "laglangyue (via GitHub)" <gi...@apache.org>.
laglangyue commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1550929993


##########
core/src/main/resources/schema/sqlserver/sqlserver-create-schema.sql:
##########
@@ -40,3 +40,24 @@ CREATE TABLE "snapshot" (
     PRIMARY KEY ("persistence_id", "sequence_number")
   )
 
+-- Create Sequence Object
+CREATE SEQUENCE global_offset
+    START WITH 1
+    INCREMENT BY 1;
+
+CREATE TABLE durable_state
+(
+    "global_offset"         BIGINT
+        CONSTRAINT [df_global_offset] DEFAULT
+        (NEXT VALUE FOR global_offset),
+    "persistence_id"        VARCHAR(255)   NOT NULL,
+    "revision"              NUMERIC(10, 0) NOT NULL,
+    "state_payload"         VARBINARY(MAX) NOT NULL,
+    "state_serial_id"       INTEGER        NOT NULL,
+    "state_serial_manifest" VARCHAR(MAX),

Review Comment:
   and pg use VARCHAR(255),it should be this?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "He-Pin (via GitHub)" <gi...@apache.org>.
He-Pin commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1550944629


##########
core/src/main/scala/org/apache/pekko/persistence/jdbc/state/SequenceNextValUpdater.scala:
##########
@@ -60,3 +60,31 @@ import slick.sql.SqlStreamingAction
 
   def getSequenceNextValueExpr() = sql"""#$nextValFetcher""".as[String]
 }
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[jdbc] class SqlServerSequenceNextValUpdater(profile: JdbcProfile,
+    val durableStateTableCfg: DurableStateTableConfiguration)
+    extends SequenceNextValUpdater {
+
+  import profile.api._
+
+  final val nextValFetcher = s"""SELECT NEXT VALUE FOR ${durableStateTableCfg.columnNames.globalOffset}""".stripMargin

Review Comment:
   sql with string template..... :(



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "Roiocam (via GitHub)" <gi...@apache.org>.
Roiocam merged PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [incubator-pekko-persistence-jdbc]

Posted by "laglangyue (via GitHub)" <gi...@apache.org>.
laglangyue commented on code in PR #158:
URL: https://github.com/apache/incubator-pekko-persistence-jdbc/pull/158#discussion_r1531726189


##########
core/src/main/scala/org/apache/pekko/persistence/jdbc/state/DurableStateQueries.scala:
##########
@@ -46,6 +46,7 @@ import pekko.persistence.jdbc.config.DurableStateTableConfiguration
   lazy val sequenceNextValUpdater = slickProfileToSchemaType(profile) match {
     case "H2"       => new H2SequenceNextValUpdater(profile, durableStateTableCfg)
     case "Postgres" => new PostgresSequenceNextValUpdater(profile, durableStateTableCfg)
+    case "MySQL"    => new MySQLSequenceNextValUpdater(profile, durableStateTableCfg)
     case _          => ???

Review Comment:
   `case _ => ???`
   Can we provide a default implementation,it is another feature
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [incubator-pekko-persistence-jdbc]

Posted by "pjfanning (via GitHub)" <gi...@apache.org>.
pjfanning commented on code in PR #158:
URL: https://github.com/apache/incubator-pekko-persistence-jdbc/pull/158#discussion_r1531899002


##########
core/src/it/scala/org/apache/pekko/persistence/jdbc/integration/ScalaJdbcDurableStateChangesByTagTest.scala:
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * license agreements; and to You under the Apache License, version 2.0:
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * This file is part of the Apache Pekko project, which was derived from Akka.

Review Comment:
   * new files should be given the standard ASF header
   * This `derived from Akka` header is just for files where an important part of the code is copied from an existing Akka file or a Pekko file that has the `derived from Akka` header.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "Roiocam (via GitHub)" <gi...@apache.org>.
Roiocam commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1551922200


##########
core/src/main/resources/schema/sqlserver/sqlserver-create-schema.sql:
##########
@@ -40,3 +40,24 @@ CREATE TABLE "snapshot" (
     PRIMARY KEY ("persistence_id", "sequence_number")
   )
 
+-- Create Sequence Object
+CREATE SEQUENCE global_offset
+    START WITH 1
+    INCREMENT BY 1;
+
+CREATE TABLE durable_state
+(
+    "global_offset"         BIGINT
+        CONSTRAINT [df_global_offset] DEFAULT
+        (NEXT VALUE FOR global_offset),
+    "persistence_id"        VARCHAR(255)   NOT NULL,
+    "revision"              NUMERIC(10, 0) NOT NULL,

Review Comment:
   Just to maintain the same column type as Event Journal.



##########
core/src/main/resources/schema/sqlserver/sqlserver-create-schema.sql:
##########
@@ -40,3 +40,24 @@ CREATE TABLE "snapshot" (
     PRIMARY KEY ("persistence_id", "sequence_number")
   )
 
+-- Create Sequence Object
+CREATE SEQUENCE global_offset
+    START WITH 1
+    INCREMENT BY 1;
+
+CREATE TABLE durable_state
+(
+    "global_offset"         BIGINT
+        CONSTRAINT [df_global_offset] DEFAULT
+        (NEXT VALUE FOR global_offset),
+    "persistence_id"        VARCHAR(255)   NOT NULL,
+    "revision"              NUMERIC(10, 0) NOT NULL,
+    "state_payload"         VARBINARY(MAX) NOT NULL,
+    "state_serial_id"       INTEGER        NOT NULL,
+    "state_serial_manifest" VARCHAR(MAX),

Review Comment:
   Same as `event_journal `, the choice of a new column needs another separate PR and more investigation.



##########
integration-test/src/test/scala/org/apache/pekko/persistence/jdbc/integration/ScalaJdbcDurableStateStoreQueryTest.scala:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.pekko.persistence.jdbc.integration
+
+import com.typesafe.config.ConfigFactory
+import org.apache.pekko
+import pekko.actor.ActorSystem
+import pekko.persistence.jdbc.state.scaladsl.JdbcDurableStateSpec
+import pekko.persistence.jdbc.testkit.internal.{ Oracle, Postgres, SchemaType, SqlServer }
+
+class PostgresScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("postgres-shared-db-application.conf"), Postgres) {
+  implicit lazy val system: ActorSystem =
+    ActorSystem("JdbcDurableStateSpec", config.withFallback(customSerializers))
+}
+
+class OracleScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("oracle-shared-db-application.conf"), Oracle) {
+  implicit lazy val system: ActorSystem =
+    ActorSystem("JdbcDurableStateSpec", config.withFallback(customSerializers))
+
+  override private[jdbc] def dropAndCreate(schemaType: SchemaType): Unit = {
+    super.dropAndCreate(schemaType)
+    withStatement(stmt => stmt.executeUpdate("""BEGIN "reset__global_offset"; END; """))
+  }
+}
+
+class SqlServerScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("sqlserver-shared-db-application.conf"), SqlServer) {
+  implicit lazy val system: ActorSystem =
+    ActorSystem("JdbcDurableStateSpec", config.withFallback(customSerializers))
+
+  override private[jdbc] def dropAndCreate(schemaType: SchemaType): Unit = {
+    super.dropAndCreate(schemaType)
+    withStatement(stmt => stmt.executeUpdate("""ALTER SEQUENCE global_offset RESTART WITH 1""".stripMargin))
+  }

Review Comment:
   That will create a new file, would this current change have a license issue?



##########
core/src/main/scala/org/apache/pekko/persistence/jdbc/state/SequenceNextValUpdater.scala:
##########
@@ -60,3 +60,31 @@ import slick.sql.SqlStreamingAction
 
   def getSequenceNextValueExpr() = sql"""#$nextValFetcher""".as[String]
 }
+
+/**
+ * INTERNAL API
+ */
+@InternalApi private[jdbc] class SqlServerSequenceNextValUpdater(profile: JdbcProfile,
+    val durableStateTableCfg: DurableStateTableConfiguration)
+    extends SequenceNextValUpdater {
+
+  import profile.api._
+
+  final val nextValFetcher = s"""SELECT NEXT VALUE FOR ${durableStateTableCfg.columnNames.globalOffset}""".stripMargin

Review Comment:
   Should it be no problem? SQL will be pre-compiled below.



##########
core/src/main/resources/schema/sqlserver/sqlserver-drop-schema.sql:
##########
@@ -1,3 +1,4 @@
 DROP TABLE IF EXISTS event_tag;
 DROP TABLE IF EXISTS event_journal;
 DROP TABLE IF EXISTS snapshot;
+DROP TABLE IF EXISTS durable_state;

Review Comment:
   That should be fine, the event_journal's SEQUENCE  doesn't have drop SQL either.



##########
integration-test/src/test/scala/org/apache/pekko/persistence/jdbc/integration/ScalaJdbcDurableStateStoreQueryTest.scala:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.pekko.persistence.jdbc.integration
+
+import com.typesafe.config.ConfigFactory
+import org.apache.pekko
+import pekko.actor.ActorSystem
+import pekko.persistence.jdbc.state.scaladsl.JdbcDurableStateSpec
+import pekko.persistence.jdbc.testkit.internal.{ Oracle, Postgres, SchemaType, SqlServer }
+
+class PostgresScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("postgres-shared-db-application.conf"), Postgres) {
+  implicit lazy val system: ActorSystem =
+    ActorSystem("JdbcDurableStateSpec", config.withFallback(customSerializers))

Review Comment:
   Fair to me.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "anchann (via GitHub)" <gi...@apache.org>.
anchann commented on PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#issuecomment-2036109415

   There's a fix for the MySQL support over in the akka repo:
   
   https://github.com/akka/akka-persistence-jdbc/pull/803
   
   What's the procedure for mirroring these kinds of fixes in the pekko repo? Are there some licensing issues? Is it just a matter of copying the change set and submitting a PR? Is there some automated process?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "laglangyue (via GitHub)" <gi...@apache.org>.
laglangyue commented on code in PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#discussion_r1545236983


##########
core/src/it/scala/org/apache/pekko/persistence/jdbc/integration/ScalaJdbcDurableStateChangesByTagTest.scala:
##########
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.pekko.persistence.jdbc.integration
+
+import com.typesafe.config.ConfigFactory
+import org.apache.pekko
+import pekko.actor.ActorSystem
+import pekko.persistence.jdbc.state.scaladsl.JdbcDurableStateSpec
+import pekko.persistence.jdbc.testkit.internal.{ MySQL, Oracle, Postgres, SqlServer }
+
+class PostgresScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("postgres-shared-db-application.conf"), Postgres) {
+  implicit lazy val system: ActorSystem =
+    ActorSystem("JdbcDurableStateSpec", config.withFallback(customSerializers))
+}
+
+class MySQLScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("mysql-shared-db-application.conf"), MySQL) {
+  implicit lazy val system: ActorSystem =
+    ActorSystem("JdbcDurableStateSpec", config.withFallback(customSerializers))
+}
+
+class OracleScalaJdbcDurableStateStoreQueryTest
+    extends JdbcDurableStateSpec(ConfigFactory.load("mysql-shared-db-application.conf"), Oracle) {

Review Comment:
   should be `oracle-shared-db-application.conf`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "Roiocam (via GitHub)" <gi...@apache.org>.
Roiocam commented on PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#issuecomment-2043017374

   > Unfortunately, the Oracle integration tests are failing due to table name issues.
   
   fixed


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org


Re: [PR] feat: durable state database support [pekko-persistence-jdbc]

Posted by "Roiocam (via GitHub)" <gi...@apache.org>.
Roiocam commented on PR #158:
URL: https://github.com/apache/pekko-persistence-jdbc/pull/158#issuecomment-2043991462

   > @Roiocam can you create an issue for following up with MySQL support - so at least, the issue is there for other users to examine and maybe provide a solution?
   
   created in #174


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@pekko.apache.org
For additional commands, e-mail: notifications-help@pekko.apache.org