|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.flink.connector.jdbc.datasource.connections.xa; |
| 19 | + |
| 20 | +import org.apache.flink.annotation.Internal; |
| 21 | +import org.apache.flink.connector.jdbc.datasource.transactions.xa.exceptions.TransientXaException; |
| 22 | +import org.apache.flink.util.function.ThrowingConsumer; |
| 23 | + |
| 24 | +import org.slf4j.Logger; |
| 25 | +import org.slf4j.LoggerFactory; |
| 26 | + |
| 27 | +import javax.annotation.Nullable; |
| 28 | +import javax.sql.XADataSource; |
| 29 | +import javax.transaction.xa.Xid; |
| 30 | + |
| 31 | +import java.io.Serializable; |
| 32 | +import java.sql.Connection; |
| 33 | +import java.sql.SQLException; |
| 34 | +import java.util.Collection; |
| 35 | +import java.util.Deque; |
| 36 | +import java.util.HashMap; |
| 37 | +import java.util.LinkedList; |
| 38 | +import java.util.Map; |
| 39 | +import java.util.function.Supplier; |
| 40 | + |
| 41 | +import static org.apache.flink.util.ExceptionUtils.rethrow; |
| 42 | +import static org.apache.flink.util.Preconditions.checkState; |
| 43 | + |
| 44 | +/** |
| 45 | + * A "pooling" implementation of {@link XaConnectionProvider}. Some database implement XA such that |
| 46 | + * one connection is limited to a single transaction. As a workaround, this implementation creates a |
| 47 | + * new XA resource after each xa_start call is made (and associates it with the xid to commit |
| 48 | + * later). |
| 49 | + */ |
| 50 | +@Internal |
| 51 | +public class PoolingXaConnectionProvider implements XaConnectionProvider { |
| 52 | + private static final long serialVersionUID = 1L; |
| 53 | + |
| 54 | + /** A supplier of connection provider. */ |
| 55 | + public interface ConnectionProviderSupplier |
| 56 | + extends Serializable, Supplier<XaConnectionProvider> {} |
| 57 | + |
| 58 | + private static final Logger LOG = LoggerFactory.getLogger(PoolingXaConnectionProvider.class); |
| 59 | + private final ConnectionProviderSupplier providerSupplier; |
| 60 | + private transient XaConnectionProvider active; |
| 61 | + private transient Map<Xid, XaConnectionProvider> mappedToXids; |
| 62 | + private transient Deque<XaConnectionProvider> pooled; |
| 63 | + |
| 64 | + public static PoolingXaConnectionProvider from( |
| 65 | + Supplier<XADataSource> dataSourceSupplier, Integer timeoutSec) { |
| 66 | + return from(() -> SimpleXaConnectionProvider.from(dataSourceSupplier, timeoutSec)); |
| 67 | + } |
| 68 | + |
| 69 | + public static PoolingXaConnectionProvider from(ConnectionProviderSupplier facadeSupplier) { |
| 70 | + return new PoolingXaConnectionProvider(facadeSupplier); |
| 71 | + } |
| 72 | + |
| 73 | + private PoolingXaConnectionProvider(ConnectionProviderSupplier facadeSupplier) { |
| 74 | + this.providerSupplier = facadeSupplier; |
| 75 | + } |
| 76 | + |
| 77 | + @Override |
| 78 | + public void open() throws Exception { |
| 79 | + checkState(active == null); |
| 80 | + pooled = new LinkedList<>(); |
| 81 | + mappedToXids = new HashMap<>(); |
| 82 | + } |
| 83 | + |
| 84 | + @Override |
| 85 | + public boolean isOpen() { |
| 86 | + return active != null && active.isOpen(); |
| 87 | + } |
| 88 | + |
| 89 | + @Override |
| 90 | + public void start(Xid xid) throws Exception { |
| 91 | + checkState(active == null); |
| 92 | + if (pooled.isEmpty()) { |
| 93 | + active = providerSupplier.get(); |
| 94 | + active.open(); |
| 95 | + } else { |
| 96 | + active = pooled.poll(); |
| 97 | + } |
| 98 | + active.start(xid); |
| 99 | + mappedToXids.put(xid, active); |
| 100 | + } |
| 101 | + |
| 102 | + /** |
| 103 | + * Must be called after {@link #start(Xid)} with the same {@link Xid}. |
| 104 | + * |
| 105 | + * @see XaConnectionProvider#endAndPrepare(Xid) |
| 106 | + */ |
| 107 | + @Override |
| 108 | + public void endAndPrepare(Xid xid) throws Exception { |
| 109 | + checkState(active == mappedToXids.get(xid)); |
| 110 | + try { |
| 111 | + active.endAndPrepare(xid); |
| 112 | + } finally { |
| 113 | + active = null; |
| 114 | + } |
| 115 | + } |
| 116 | + |
| 117 | + @Override |
| 118 | + public void commit(Xid xid, boolean ignoreUnknown) throws TransientXaException { |
| 119 | + runForXid(xid, facade -> facade.commit(xid, ignoreUnknown)); |
| 120 | + } |
| 121 | + |
| 122 | + @Override |
| 123 | + public void rollback(Xid xid) throws TransientXaException { |
| 124 | + runForXid(xid, facade -> facade.rollback(xid)); |
| 125 | + } |
| 126 | + |
| 127 | + @Override |
| 128 | + public void failAndRollback(Xid xid) throws TransientXaException { |
| 129 | + runForXid(xid, facade -> facade.failAndRollback(xid)); |
| 130 | + } |
| 131 | + |
| 132 | + @Override |
| 133 | + public Collection<Xid> recover() throws TransientXaException { |
| 134 | + return peekPooled().recover(); |
| 135 | + } |
| 136 | + |
| 137 | + @Override |
| 138 | + public void close() throws Exception { |
| 139 | + for (XaConnectionProvider facade : mappedToXids.values()) { |
| 140 | + facade.close(); |
| 141 | + } |
| 142 | + for (XaConnectionProvider facade : pooled) { |
| 143 | + facade.close(); |
| 144 | + } |
| 145 | + if (active != null && active.isOpen()) { |
| 146 | + active.close(); |
| 147 | + } |
| 148 | + } |
| 149 | + |
| 150 | + @Nullable |
| 151 | + @Override |
| 152 | + public Connection getConnection() { |
| 153 | + return active.getConnection(); |
| 154 | + } |
| 155 | + |
| 156 | + @Override |
| 157 | + public boolean isConnectionValid() throws SQLException { |
| 158 | + return active.isConnectionValid(); |
| 159 | + } |
| 160 | + |
| 161 | + @Override |
| 162 | + public Connection getOrEstablishConnection() throws SQLException, ClassNotFoundException { |
| 163 | + return active.getOrEstablishConnection(); |
| 164 | + } |
| 165 | + |
| 166 | + @Override |
| 167 | + public void closeConnection() { |
| 168 | + active.closeConnection(); |
| 169 | + } |
| 170 | + |
| 171 | + @Override |
| 172 | + public Connection reestablishConnection() throws SQLException, ClassNotFoundException { |
| 173 | + return active.reestablishConnection(); |
| 174 | + } |
| 175 | + |
| 176 | + // WARN: action MUST leave the facade in IDLE state (i.e. not start/end/prepare any tx) |
| 177 | + private void runForXid( |
| 178 | + Xid xid, ThrowingConsumer<XaConnectionProvider, TransientXaException> action) { |
| 179 | + XaConnectionProvider mapped = mappedToXids.remove(xid); |
| 180 | + if (mapped == null) { |
| 181 | + // a transaction can be not known during recovery |
| 182 | + LOG.debug("No XA resource found associated with XID: {}", xid); |
| 183 | + action.accept(peekPooled()); |
| 184 | + } else { |
| 185 | + LOG.debug("Found mapped XA resource for XID: {} {}", xid, mapped); |
| 186 | + try { |
| 187 | + action.accept(mapped); |
| 188 | + } finally { |
| 189 | + pooled.offer(mapped); |
| 190 | + } |
| 191 | + } |
| 192 | + } |
| 193 | + |
| 194 | + // WARN: the returned facade MUST be left in IDLE state (i.e. not start/end/prepare any tx) |
| 195 | + private XaConnectionProvider peekPooled() { |
| 196 | + XaConnectionProvider xaFacade = pooled.peek(); |
| 197 | + if (xaFacade == null) { |
| 198 | + xaFacade = providerSupplier.get(); |
| 199 | + try { |
| 200 | + xaFacade.open(); |
| 201 | + } catch (Exception e) { |
| 202 | + rethrow(e); |
| 203 | + } |
| 204 | + pooled.offer(xaFacade); |
| 205 | + } |
| 206 | + return xaFacade; |
| 207 | + } |
| 208 | +} |
0 commit comments