Uploaded image for project: 'Calcite'
  1. Calcite
  2. CALCITE-2882

Connection properties are lost after timeout

Details

    Description

      When use avatica connect Phoenix QueryServer, I create an AvaticaConnection:

      Connection conntion = DriverManage.getConnection(url);
      connection.setAutoCommit(true);
      

      Avatica keep PhoenixConnection alive in the Cache, which will be expired after 10min by default.
      I still use the older AvaticaConnection , it will reopen an PhoenixConnection, but the ConnectionProperties is loss, such as AutoCommit.

      I use sqlline-thin.py to reappear the problem:
      1) sqlline-thin.py http://localhost:8765
      2) upsert one row and select

      3) after 10 min, upsert again, the connection will be recreate, but select null

      link title

      Attachments

        1. image-2019-02-28-17-25-39-478.png
          88 kB
          shining
        2. image-2019-02-28-17-28-31-926.png
          417 kB
          shining

        Issue Links

          Activity

            githubbot ASF GitHub Bot logged work - 17/Apr/19 23:49
            • Time Spent:
              10m
               
              zabetak commented on pull request #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86
               
               
                 
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 16/Apr/19 08:31
            • Time Spent:
              10m
               
              bakea commented on pull request #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#discussion_r275655362
               
               

               ##########
               File path: server/src/test/java/org/apache/calcite/avatica/remote/AvaticaServersForTest.java
               ##########
               @@ -164,7 +190,20 @@ static JdbcMeta getInstance() {
                     if (instance == null) {
                       try {
                         instance = new JdbcMeta(CONNECTION_SPEC.url, CONNECTION_SPEC.username,
              - CONNECTION_SPEC.password);
              + CONNECTION_SPEC.password);
              + } catch (SQLException e) {
              + throw new RuntimeException(e);
              + }
              + }
              + return instance;
              + }
              +
              + static JdbcMeta getInstance(Properties properties) {
              + if (instance == null) {
               
               Review comment:
                 Good suggestion, I made a mistake.
                 I new a `PropertyRemoteJdbcMetaFactory` to handle instance with `Properties` instead of `FullyRemoteJdbcMetaFactory`.
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 16/Apr/19 07:09
            • Time Spent:
              10m
               
              bakea commented on pull request #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#discussion_r275655833
               
               

               ##########
               File path: server/src/test/java/org/apache/calcite/avatica/remote/ConnectionPropertiesTest.java
               ##########
               @@ -0,0 +1,108 @@
              +/*
              + * 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.calcite.avatica.remote;
              +
              +import org.apache.calcite.avatica.AvaticaConnection;
              +import org.apache.calcite.avatica.ConnectionSpec;
              +import org.apache.calcite.avatica.jdbc.JdbcMeta;
              +import org.apache.calcite.avatica.server.HttpServer;
              +
              +import com.google.common.cache.Cache;
              +
              +import org.junit.Test;
              +import org.junit.runner.RunWith;
              +import org.junit.runners.Parameterized;
              +
              +import java.lang.reflect.Field;
              +import java.sql.Connection;
              +import java.sql.DriverManager;
              +import java.util.List;
              +import java.util.Properties;
              +import java.util.concurrent.TimeUnit;
              +
              +import static org.junit.Assert.assertEquals;
              +import static org.junit.Assert.assertFalse;
              +
              +/** Tests covering {@link ConnectionPropertiesTest}. */
              +@RunWith(Parameterized.class)
              +public class ConnectionPropertiesTest {
              + private static final AvaticaServersForTest SERVERS = new AvaticaServersForTest();
              +
              + private final HttpServer server;
              + private final String url;
              + private final int port;
              + private final Driver.Serialization serialization;
              +
              + @Parameterized.Parameters(name = "{0}")
              + public static List<Object[]> parameters() throws Exception {
              + Properties prop = new Properties();
              + prop.put(JdbcMeta.ConnectionCacheSettings.EXPIRY_DURATION.key(), "1");
              + prop.put(JdbcMeta.ConnectionCacheSettings.EXPIRY_UNIT.key(), TimeUnit.SECONDS.name());
              + SERVERS.startServers(prop);
              + return SERVERS.getJUnitParameters();
              + }
              +
              + public ConnectionPropertiesTest(Driver.Serialization serialization,
              + HttpServer server) {
              + this.server = server;
              + this.port = this.server.getPort();
              + this.serialization = serialization;
              + this.url = SERVERS.getJdbcUrl(port, serialization);
              + }
              +
              + @Test
              + public void testConnectionPropertiesSync() throws Exception {
              + ConnectionSpec.getDatabaseLock().lock();
              + try {
              + AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url);
              + conn.setAutoCommit(false);
              + conn.setTransactionIsolation(Connection.TRANSACTION_REPEATABLE_READ);
              +
              + // sync connection properties
              + conn.createStatement();
              + Connection remoteConn = getConnection(
              + AvaticaServersForTest.FullyRemoteJdbcMetaFactory.getInstance(), conn.id);
              +
              + assertFalse(remoteConn.getAutoCommit());
              + assertEquals(remoteConn.getTransactionIsolation(),
              + Connection.TRANSACTION_REPEATABLE_READ);
              +
              + // after 1s, remote connection expired and reopen
              + Thread.sleep(1000);
              +
              + conn.createStatement();
              + Connection remoteConn1 = getConnection(
              + AvaticaServersForTest.FullyRemoteJdbcMetaFactory.getInstance(), conn.id);
              +
              + assertFalse(remoteConn1.getAutoCommit());
              + assertEquals(remoteConn1.getTransactionIsolation(),
              + Connection.TRANSACTION_REPEATABLE_READ);
              + } finally {
              + ConnectionSpec.getDatabaseLock().unlock();
              + }
              + }
              +
              + private static Connection getConnection(JdbcMeta m, String id) throws Exception {
              + Field f = JdbcMeta.class.getDeclaredField("connectionCache");
               
               Review comment:
                 The test case isnot under the same package with JdbcMeta, So I use refleaction.
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 16/Apr/19 07:07
            • Time Spent:
              10m
               
              bakea commented on pull request #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#discussion_r275655362
               
               

               ##########
               File path: server/src/test/java/org/apache/calcite/avatica/remote/AvaticaServersForTest.java
               ##########
               @@ -164,7 +190,20 @@ static JdbcMeta getInstance() {
                     if (instance == null) {
                       try {
                         instance = new JdbcMeta(CONNECTION_SPEC.url, CONNECTION_SPEC.username,
              - CONNECTION_SPEC.password);
              + CONNECTION_SPEC.password);
              + } catch (SQLException e) {
              + throw new RuntimeException(e);
              + }
              + }
              + return instance;
              + }
              +
              + static JdbcMeta getInstance(Properties properties) {
              + if (instance == null) {
               
               Review comment:
                 Good suggestion, I made a mistake. But I cannot find better solution, so I new a `JdbcMeta` instance in startServers method without `FullyRemoteJdbcMetaFactory`.
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 16/Apr/19 07:00
            • Time Spent:
              10m
               
              bakea commented on pull request #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#discussion_r275653444
               
               

               ##########
               File path: server/src/test/java/org/apache/calcite/avatica/remote/AvaticaServersForTest.java
               ##########
               @@ -70,6 +71,31 @@ public void startServers() throws Exception {
                   serversBySerialization.put(Serialization.PROTOBUF, protobufServer);
                 }
               
              + /**
              + * Starts Avatica servers for each serialization type with the properties.
              + */
              + public void startServers(Properties properties) {
               
               Review comment:
                 yeah! That looks more normalized.
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 15/Apr/19 22:02
            • Time Spent:
              10m
               
              zabetak commented on pull request #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#discussion_r275561238
               
               

               ##########
               File path: server/src/test/java/org/apache/calcite/avatica/remote/ConnectionPropertiesTest.java
               ##########
               @@ -0,0 +1,108 @@
              +/*
              + * 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.calcite.avatica.remote;
              +
              +import org.apache.calcite.avatica.AvaticaConnection;
              +import org.apache.calcite.avatica.ConnectionSpec;
              +import org.apache.calcite.avatica.jdbc.JdbcMeta;
              +import org.apache.calcite.avatica.server.HttpServer;
              +
              +import com.google.common.cache.Cache;
              +
              +import org.junit.Test;
              +import org.junit.runner.RunWith;
              +import org.junit.runners.Parameterized;
              +
              +import java.lang.reflect.Field;
              +import java.sql.Connection;
              +import java.sql.DriverManager;
              +import java.util.List;
              +import java.util.Properties;
              +import java.util.concurrent.TimeUnit;
              +
              +import static org.junit.Assert.assertEquals;
              +import static org.junit.Assert.assertFalse;
              +
              +/** Tests covering {@link ConnectionPropertiesTest}. */
              +@RunWith(Parameterized.class)
              +public class ConnectionPropertiesTest {
              + private static final AvaticaServersForTest SERVERS = new AvaticaServersForTest();
              +
              + private final HttpServer server;
              + private final String url;
              + private final int port;
              + private final Driver.Serialization serialization;
              +
              + @Parameterized.Parameters(name = "{0}")
              + public static List<Object[]> parameters() throws Exception {
              + Properties prop = new Properties();
              + prop.put(JdbcMeta.ConnectionCacheSettings.EXPIRY_DURATION.key(), "1");
              + prop.put(JdbcMeta.ConnectionCacheSettings.EXPIRY_UNIT.key(), TimeUnit.SECONDS.name());
              + SERVERS.startServers(prop);
              + return SERVERS.getJUnitParameters();
              + }
              +
              + public ConnectionPropertiesTest(Driver.Serialization serialization,
              + HttpServer server) {
              + this.server = server;
              + this.port = this.server.getPort();
              + this.serialization = serialization;
              + this.url = SERVERS.getJdbcUrl(port, serialization);
              + }
              +
              + @Test
              + public void testConnectionPropertiesSync() throws Exception {
              + ConnectionSpec.getDatabaseLock().lock();
              + try {
              + AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url);
              + conn.setAutoCommit(false);
              + conn.setTransactionIsolation(Connection.TRANSACTION_REPEATABLE_READ);
              +
              + // sync connection properties
              + conn.createStatement();
              + Connection remoteConn = getConnection(
              + AvaticaServersForTest.FullyRemoteJdbcMetaFactory.getInstance(), conn.id);
              +
              + assertFalse(remoteConn.getAutoCommit());
              + assertEquals(remoteConn.getTransactionIsolation(),
              + Connection.TRANSACTION_REPEATABLE_READ);
              +
              + // after 1s, remote connection expired and reopen
              + Thread.sleep(1000);
              +
              + conn.createStatement();
              + Connection remoteConn1 = getConnection(
              + AvaticaServersForTest.FullyRemoteJdbcMetaFactory.getInstance(), conn.id);
              +
              + assertFalse(remoteConn1.getAutoCommit());
              + assertEquals(remoteConn1.getTransactionIsolation(),
              + Connection.TRANSACTION_REPEATABLE_READ);
              + } finally {
              + ConnectionSpec.getDatabaseLock().unlock();
              + }
              + }
              +
              + private static Connection getConnection(JdbcMeta m, String id) throws Exception {
              + Field f = JdbcMeta.class.getDeclaredField("connectionCache");
               
               Review comment:
                 Is there really need for reflection? The `JdbcMeta` class has a `protected` method `getConnectionCache`. If the test is under the same package you can use the method directly.
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 15/Apr/19 22:02
            • Time Spent:
              10m
               
              zabetak commented on pull request #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#discussion_r275232463
               
               

               ##########
               File path: server/src/test/java/org/apache/calcite/avatica/remote/AvaticaServersForTest.java
               ##########
               @@ -164,7 +190,20 @@ static JdbcMeta getInstance() {
                     if (instance == null) {
                       try {
                         instance = new JdbcMeta(CONNECTION_SPEC.url, CONNECTION_SPEC.username,
              - CONNECTION_SPEC.password);
              + CONNECTION_SPEC.password);
              + } catch (SQLException e) {
              + throw new RuntimeException(e);
              + }
              + }
              + return instance;
              + }
              +
              + static JdbcMeta getInstance(Properties properties) {
              + if (instance == null) {
               
               Review comment:
                 If `instance` is already initialised then this method will not take into account the properties which means that test relying on the properties will be broken.
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 15/Apr/19 22:02
            • Time Spent:
              10m
               
              zabetak commented on pull request #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#discussion_r275559518
               
               

               ##########
               File path: server/src/test/java/org/apache/calcite/avatica/remote/AvaticaServersForTest.java
               ##########
               @@ -70,6 +71,31 @@ public void startServers() throws Exception {
                   serversBySerialization.put(Serialization.PROTOBUF, protobufServer);
                 }
               
              + /**
              + * Starts Avatica servers for each serialization type with the properties.
              + */
              + public void startServers(Properties properties) {
               
               Review comment:
                 The method is almost the same with the one below. How about a small refactoring?
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 15/Apr/19 01:30
            • Time Spent:
              10m
               
              bakea commented on pull request #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#discussion_r275186546
               
               

               ##########
               File path: server/src/test/java/org/apache/calcite/avatica/remote/AvaticaServersForTest.java
               ##########
               @@ -163,8 +165,12 @@ public String getJdbcUrl(int port, Serialization serialization, String urlSuffix
                   static JdbcMeta getInstance() {
                     if (instance == null) {
                       try {
              - instance = new JdbcMeta(CONNECTION_SPEC.url, CONNECTION_SPEC.username,
              - CONNECTION_SPEC.password);
              + Properties info = new Properties();
              + info.put(JdbcMeta.ConnectionCacheSettings.EXPIRY_DURATION.key(), "10");
              + info.put(JdbcMeta.ConnectionCacheSettings.EXPIRY_UNIT.key(), TimeUnit.SECONDS.name());
               
               Review comment:
                 ok.I will do that.THX
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 12/Apr/19 11:50
            • Time Spent:
              10m
               
              danny0405 commented on pull request #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#discussion_r274872221
               
               

               ##########
               File path: server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
               ##########
               @@ -713,6 +713,40 @@ private int getSales(Connection conn, String salesTable, int productId) throws E
                     assertEquals(props, originalProps);
                   }
                 }
              +
              + @Test public void testConnectionPropertiesSync() throws Exception {
              + ConnectionSpec.getDatabaseLock().lock();
              + try {
              + final String t = AvaticaUtils.unique("TEST_CONNECTION_PROPERTIES");
              + AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url);
              + conn.setAutoCommit(false);
              +
              + Statement statement = conn.createStatement();
              + final String create =
              + String.format(Locale.ROOT, "create table if not exists %s ("
              + + " id int not null, msg varchar(255) not null)", t);
              + int status = statement.executeUpdate(create);
              + assertEquals(status, 0);
              +
              + Thread.sleep(10000);
              + statement = conn.createStatement();
               
               Review comment:
                 That really sleep too much time, better to config this timeout params just in this case and make it fast enough, if it is much slower, maybe you should add this test case in [1].
                 [1] https://github.com/apache/calcite/blob/af97f55f1fcbb3cf6005393dffd91eb345c04b6f/core/src/test/java/org/apache/calcite/test/CalciteSuite.java#L19
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 12/Apr/19 11:47
            • Time Spent:
              10m
               
              danny0405 commented on pull request #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#discussion_r274871459
               
               

               ##########
               File path: server/src/test/java/org/apache/calcite/avatica/remote/AvaticaServersForTest.java
               ##########
               @@ -163,8 +165,12 @@ public String getJdbcUrl(int port, Serialization serialization, String urlSuffix
                   static JdbcMeta getInstance() {
                     if (instance == null) {
                       try {
              - instance = new JdbcMeta(CONNECTION_SPEC.url, CONNECTION_SPEC.username,
              - CONNECTION_SPEC.password);
              + Properties info = new Properties();
              + info.put(JdbcMeta.ConnectionCacheSettings.EXPIRY_DURATION.key(), "10");
              + info.put(JdbcMeta.ConnectionCacheSettings.EXPIRY_UNIT.key(), TimeUnit.SECONDS.name());
               
               Review comment:
                 Does this expire time config has any side effect with the test already ? I'm a little worry about it, maybe you can make these 2 params configurable and set them in your test case.
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 09/Apr/19 10:03
            • Time Spent:
              10m
               
              bakea commented on issue #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#issuecomment-481187717
               
               
                 > I went through the JIRA case, I get what's the problem, but I don't get why and where it occurs. Without a test is difficult to verify that the fix is valid.
                 
                 @zabetak I add a test for this problem, if dirty set false after connProperties sync, although I set autocommit false, when connection reopen, autocommit is still effect,data still be queried.
                 If dirty initialize true when open connection everytime,query nothing when connection reopen.
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 09/Apr/19 05:32
            • Time Spent:
              10m
               
              bakea commented on issue #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#issuecomment-481108356
               
               
                 > I don't think it is by design that we are automatically re-creating a new connection for you after your previous connection was closed due to lack of use. What is triggering the re-connection that you refer to?
                 
                 I find this in phoenix queryserver, i initialize a avaticaConnection which cache an phoenixConnection in the queryserver. If avaticaConnection idle exceed the phoenixConnection cache keepalive time, it will be reopen a phoenixConnection when i use it again.
                 But the param dirty is false, connectionPropertis cannot be sync to phoenixconnection such as autocommit...
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 08/Apr/19 22:56
            • Time Spent:
              10m
               
              zabetak commented on issue #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#issuecomment-481037599
               
               
                 I went through the JIRA case, I get what's the problem, but I don't get why and where it occurs. Without a test is difficult to verify that the fix is valid.
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 04/Apr/19 14:22
            • Time Spent:
              10m
               
              joshelser commented on issue #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#issuecomment-479918109
               
               
                 I don't think it is by design that we are automatically re-creating a new connection for you after your previous connection was closed due to lack of use. What is triggering the re-connection that you refer to?
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 02/Apr/19 02:45
            • Time Spent:
              10m
               
              F21 commented on issue #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#issuecomment-478822344
               
               
                 @bakea Do you think it would be possible to add a test for 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.
               
              For queries about this service, please contact Infrastructure at:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 07/Mar/19 02:13
            • Time Spent:
              10m
               
              bakea commented on issue #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86#issuecomment-470355556
               
               
                 Someone review it?
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org
            githubbot ASF GitHub Bot logged work - 02/Mar/19 07:30
            • Time Spent:
              10m
               
              bakea commented on pull request #86: [CALCITE-2882] Set dirty true when open connection
              URL: https://github.com/apache/calcite-avatica/pull/86
               
               
                 Set dirty false in connectionPropertis in order to peoperties can be sync when createConnection again
               
              ----------------------------------------------------------------
              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:
              users@infra.apache.org

            People

              Unassigned Unassigned
              shining shining
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 2h 50m
                  2h 50m