curcur commented on a change in pull request #15636:
URL: https://github.com/apache/flink/pull/15636#discussion_r615231118



##########
File path: 
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/xa/XaFacadePoolingImpl.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.flink.connector.jdbc.xa;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.transaction.xa.Xid;
+
+import java.io.Serializable;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.function.Supplier;
+
+import static org.apache.flink.util.ExceptionUtils.rethrow;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A "pooling" implementation of {@link XaFacade}. Some database implement XA 
such that one
+ * connection is limited to a single transaction. As a workaround, this 
implementation creates a new
+ * XA resource after each xa_prepare call is made (and associates the current 
one with the xid to
+ * commit later).
+ */
+@Internal
+class XaFacadePoolingImpl implements XaFacade {
+    private static final long serialVersionUID = 1L;
+
+    public interface FacadeSupplier extends Serializable, Supplier<XaFacade> {}
+
+    private static final transient Logger LOG = 
LoggerFactory.getLogger(XaFacadePoolingImpl.class);
+    private final FacadeSupplier facadeSupplier;
+    private transient XaFacade active;
+    private transient Map<Xid, XaFacade> prepared;
+    private transient Deque<XaFacade> pooled;

Review comment:
       make these two final and initialize them in the constructor instead?

##########
File path: 
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkFunction.java
##########
@@ -286,12 +287,15 @@ private void prepareCurrentTx(long checkpointId) throws 
IOException {
         outputFormat.flush();
         try {
             xaFacade.endAndPrepare(currentXid);
+            outputFormat.reconnect(false);

Review comment:
       Is this the fix for FLINK-22311?

##########
File path: 
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/xa/XaFacadePoolingImpl.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.flink.connector.jdbc.xa;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.transaction.xa.Xid;
+
+import java.io.Serializable;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.function.Supplier;
+
+import static org.apache.flink.util.ExceptionUtils.rethrow;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A "pooling" implementation of {@link XaFacade}. Some database implement XA 
such that one
+ * connection is limited to a single transaction. As a workaround, this 
implementation creates a new
+ * XA resource after each xa_prepare call is made (and associates the current 
one with the xid to
+ * commit later).
+ */
+@Internal
+class XaFacadePoolingImpl implements XaFacade {
+    private static final long serialVersionUID = 1L;
+
+    public interface FacadeSupplier extends Serializable, Supplier<XaFacade> {}
+
+    private static final transient Logger LOG = 
LoggerFactory.getLogger(XaFacadePoolingImpl.class);
+    private final FacadeSupplier facadeSupplier;
+    private transient XaFacade active;
+    private transient Map<Xid, XaFacade> prepared;
+    private transient Deque<XaFacade> pooled;
+
+    XaFacadePoolingImpl(FacadeSupplier facadeSupplier) {
+        this.facadeSupplier = facadeSupplier;
+    }
+
+    @Override
+    public void open() throws Exception {
+        checkState(active == null);
+        active = facadeSupplier.get();
+        active.open();
+        pooled = new LinkedList<>();
+        prepared = new HashMap<>();
+    }
+
+    @Override
+    public boolean isOpen() {
+        return active != null && active.isOpen();
+    }
+
+    @Override
+    public void start(Xid xid) throws TransientXaException {
+        active.start(xid);
+    }

Review comment:
       Logically, I was wondering whether to put the following "new/recycle 
connection" to the `start` method, which may be more error-prune and 
straightforward (of course, the code needs some adjustment).
   
   ```
   if (pooled.isEmpty()) {
               active = facadeSupplier.get();
               active.open();
           } else {
               active = pooled.poll();
           }
   ```
   
   The reason is as follows:
   `start()` is called when a new trans begins, that's where a new idle 
connection is needed.
   
   Where current implementation has an implicit assumption that `start()` is 
always called after `endAndPrepare()`, which is true according to how this is 
implemented in `JdbcXaSinkFunction.snapshotState()`.
   
   But I am not sure whether we should have such implicit assumptions by 
default.
   
   

##########
File path: 
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/xa/XaFacadeImpl.java
##########
@@ -82,12 +82,12 @@
 
     /** @return a non-serializable instance. */

Review comment:
       same as above

##########
File path: 
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/xa/XaFacadePoolingImpl.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.flink.connector.jdbc.xa;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.transaction.xa.Xid;
+
+import java.io.Serializable;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.function.Supplier;
+
+import static org.apache.flink.util.ExceptionUtils.rethrow;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A "pooling" implementation of {@link XaFacade}. Some database implement XA 
such that one
+ * connection is limited to a single transaction. As a workaround, this 
implementation creates a new
+ * XA resource after each xa_prepare call is made (and associates the current 
one with the xid to
+ * commit later).
+ */
+@Internal
+class XaFacadePoolingImpl implements XaFacade {
+    private static final long serialVersionUID = 1L;
+
+    public interface FacadeSupplier extends Serializable, Supplier<XaFacade> {}
+
+    private static final transient Logger LOG = 
LoggerFactory.getLogger(XaFacadePoolingImpl.class);
+    private final FacadeSupplier facadeSupplier;
+    private transient XaFacade active;
+    private transient Map<Xid, XaFacade> prepared;
+    private transient Deque<XaFacade> pooled;
+
+    XaFacadePoolingImpl(FacadeSupplier facadeSupplier) {
+        this.facadeSupplier = facadeSupplier;
+    }
+
+    @Override
+    public void open() throws Exception {
+        checkState(active == null);
+        active = facadeSupplier.get();
+        active.open();
+        pooled = new LinkedList<>();
+        prepared = new HashMap<>();
+    }
+
+    @Override
+    public boolean isOpen() {
+        return active != null && active.isOpen();
+    }
+
+    @Override
+    public void start(Xid xid) throws TransientXaException {
+        active.start(xid);
+    }
+
+    @Override
+    public void endAndPrepare(Xid xid) throws Exception {
+        checkState(!prepared.containsKey(xid));
+        active.endAndPrepare(xid);
+        prepared.put(xid, active);
+        if (pooled.isEmpty()) {
+            active = facadeSupplier.get();
+            active.open();
+        } else {
+            active = pooled.poll();
+        }
+    }
+
+    @Override
+    public void commit(Xid xid, boolean ignoreUnknown) throws 
TransientXaException {
+        runForXid(xid, facade -> facade.commit(xid, ignoreUnknown));
+    }
+
+    @Override
+    public void rollback(Xid xid) throws TransientXaException {
+        runForXid(xid, facade -> facade.rollback(xid));
+    }
+
+    @Override
+    public void failOrRollback(Xid xid) throws TransientXaException {
+        runForXid(xid, facade -> facade.failOrRollback(xid));
+    }
+
+    @Override
+    public Collection<Xid> recover() throws TransientXaException {
+        return peekPooled().recover();
+    }
+
+    @Override
+    public void close() throws Exception {
+        active.close();
+        for (XaFacade facade : prepared.values()) {
+            facade.close();
+        }
+        for (XaFacade facade : pooled) {
+            facade.close();
+        }
+    }
+
+    @Nullable
+    @Override
+    public Connection getConnection() {
+        return active.getConnection();
+    }
+
+    @Override
+    public boolean isConnectionValid() throws SQLException {
+        return active.isConnectionValid();
+    }
+
+    @Override
+    public Connection getOrEstablishConnection() throws SQLException, 
ClassNotFoundException {
+        return active.getOrEstablishConnection();
+    }
+
+    @Override
+    public void closeConnection() {
+        active.closeConnection();
+    }
+
+    @Override
+    public Connection reestablishConnection() throws SQLException, 
ClassNotFoundException {
+        return active.reestablishConnection();
+    }
+
+    // WARN: action MUST leave the facade in IDLE state (i.e. not 
start/end/prepare any tx)
+    private void runForXid(Xid xid, ThrowingConsumer<XaFacade, 
TransientXaException> action) {
+        XaFacade mapped = prepared.remove(xid);
+        if (mapped == null) {
+            LOG.debug("No XA resource found associated with XID: {}", xid);
+            action.accept(peekPooled());

Review comment:
       when this would happen?
   
   that says we have a prepared trans to commit/rollback/fail without a 
connection associated with it.
   
   During the recovery I guess.
   
   Document it a bit?
   

##########
File path: 
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/xa/XaFacade.java
##########
@@ -52,8 +51,8 @@
 
     /** @return a non-serializable instance. */

Review comment:
       This is only for test?
   
   Mark as visible for test?
   

##########
File path: 
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/xa/XaFacade.java
##########
@@ -52,8 +51,8 @@
 
     /** @return a non-serializable instance. */

Review comment:
       and other similar places.




-- 
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.

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


Reply via email to