You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@phoenix.apache.org by kadirozde <gi...@git.apache.org> on 2018/09/27 15:12:00 UTC

[GitHub] phoenix pull request #359: Phoenix 4764 - Cleanup metadata of child views fo...

GitHub user kadirozde opened a pull request:

    https://github.com/apache/phoenix/pull/359

    Phoenix 4764 - Cleanup metadata of child views for a base table that has been dropped

    A given Phoenix table may have one or (child) views and these views may have their own views. So, a given table or view may have a tree of views rooted at this given table or view. When a table or view is deleted, its views are also required to be deleted. Given that there is no transaction mechanism to atomically delete a given table and its view, failures during deleting a table or view will lead to orphaned views. This patch is to make sure that all the child views will be dropped and will not stay around as orphan views after their parent is deleted.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/kadirozde/phoenix PHOENIX-4764

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/phoenix/pull/359.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #359
    
----
commit cb230037a35fce89c8d9ea24a523e3ba48ece672
Author: Kadir <ko...@...>
Date:   2018-09-27T06:32:31Z

    Initial commit

commit 3ff6d5f1973cc42e29d0534e618c2b674ffa95ce
Author: Kadir <ko...@...>
Date:   2018-09-27T14:46:13Z

    Initial changes on the existing files

----


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r221773024
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.cache.GlobalCache;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +        GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().invalidateAll();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        // sleep a little bit to compensate time clock skew when SYSTEM.CATALOG moves
    +        // among region servers because we relies on server time of RS which is hosting
    +        // SYSTEM.CATALOG
    +        Configuration config = env.getConfiguration();
    --- End diff --
    
    Same here, is this really needed?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222448618
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    The test looks pretty big. Do you want it to split this into multiple tests or move some code into `before`and `after` test methods?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222444777
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -357,18 +357,28 @@
     			+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
     	
    -	 public static final String CREATE_MUTEX_METADTA =
    -	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    -	            // Pk columns
    -	            TENANT_ID + " VARCHAR NULL," +
    -	            TABLE_SCHEM + " VARCHAR NULL," +
    -	            TABLE_NAME + " VARCHAR NOT NULL," +
    -	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    -	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    -	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    -	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    -	            HConstants.VERSIONS + "=%s,\n" +
    -	            ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    -	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    -    
    +	public static final String CREATE_MUTEX_METADTA =
    +            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +             // Pk columns
    +            TENANT_ID + " VARCHAR NULL," +
    +            TABLE_SCHEM + " VARCHAR NULL," +
    +	        TABLE_NAME + " VARCHAR NOT NULL," +
    +	        COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	        COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	        "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	        + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	        HConstants.VERSIONS + "=%s,\n" +
    +	        ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    +	        PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    +
    +	public static final String CREATE_TASK_METADATA = "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\""
    +            + SYSTEM_TASK_TABLE + "\"(\n" +
    +            // PK columns
    +            TASK_TYPE + " UNSIGNED_TINYINT NOT NULL," + TASK_TS + " TIMESTAMP NOT NULL," + TENANT_ID + " VARCHAR NULL," + TABLE_SCHEM + " VARCHAR NULL," +
    --- End diff --
    
    `UNSIGNED_TINYINT` for `TASK_TYPE` -- Any particular reason for it? What is max value here?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223779871
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    --- End diff --
    
    The initial values for these private variables are given in the constructor method. Please see phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java where I supply new values for these parameters.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223910162
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -2655,10 +2666,19 @@ private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] s
                         boolean hasChildViews =
                                 ViewFinder.hasChildViews(hTable, tenantId, schemaName, tableName,
                                     clientTimeStamp);
    -                    if (hasChildViews && !isCascade) {
    -                        // DROP without CASCADE on tables with child views is not permitted
    -                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
    -                                EnvironmentEdgeManager.currentTimeMillis(), null);
    +                    if (hasChildViews) {
    +                        if (!isCascade) {
    +                            // DROP without CASCADE on tables with child views is not permitted
    --- End diff --
    
    Drop without specifying CASCADE will fail if the table has child views, so that the user knows that   child views exist on the table. If the user wants to drop a table and child views they can just used DROP CASCADE. 


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223779980
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.cache.GlobalCache;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +        GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().invalidateAll();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        // sleep a little bit to compensate time clock skew when SYSTEM.CATALOG moves
    +        // among region servers because we relies on server time of RS which is hosting
    +        // SYSTEM.CATALOG
    +        Configuration config = env.getConfiguration();
    --- End diff --
    
    I am following the implementation of MetaDataRegionObserver. I assume the check is not needed here as it was not needed for MetaDataRegionObserver either.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223886025
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    I don't think its possible to split this test, it just creates some views, drops the parent and then verifies that our process eventually cleans up the view metadata.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223780489
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, String tenantId, String schemaName, String tableName)
    +            throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME + " (" +
    +                    PhoenixDatabaseMetaData.TASK_TYPE + ", " +
    +                    PhoenixDatabaseMetaData.TENANT_ID + ", " +
    +                    PhoenixDatabaseMetaData.TABLE_SCHEM + ", " +
    +                    PhoenixDatabaseMetaData.TABLE_NAME + ") VALUES(?,?,?,?)");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            if (tenantId != null) {
    +                stmt.setString(2, tenantId);
    +            }else {
    --- End diff --
    
    Done


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r221776267
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -357,18 +357,28 @@
     			+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
     	
    -	 public static final String CREATE_MUTEX_METADTA =
    -	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    -	            // Pk columns
    -	            TENANT_ID + " VARCHAR NULL," +
    -	            TABLE_SCHEM + " VARCHAR NULL," +
    -	            TABLE_NAME + " VARCHAR NOT NULL," +
    -	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    -	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    -	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    -	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    -	            HConstants.VERSIONS + "=%s,\n" +
    -	            ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    -	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    -    
    +	public static final String CREATE_MUTEX_METADTA =
    +            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +             // Pk columns
    +            TENANT_ID + " VARCHAR NULL," +
    +            TABLE_SCHEM + " VARCHAR NULL," +
    +	        TABLE_NAME + " VARCHAR NOT NULL," +
    +	        COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	        COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	        "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	        + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	        HConstants.VERSIONS + "=%s,\n" +
    +	        ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    +	        PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    +
    +	public static final String CREATE_TASK_METADATA = "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\""
    +            + SYSTEM_TASK_TABLE + "\"(\n" +
    +            // PK columns
    +            TASK_TYPE + " UNSIGNED_TINYINT NOT NULL," + TASK_TS + " TIMESTAMP NOT NULL," + TENANT_ID + " VARCHAR NULL," + TABLE_SCHEM + " VARCHAR NULL," +
    --- End diff --
    
    You could use the ROW_TIMESTAMP feature to let phoenix set the timestamp automatically, this would speed up queries if there are lots of rows in this table (http://phoenix.apache.org/rowtimestamp.html)


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223898473
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java ---
    @@ -540,6 +540,12 @@ public static boolean isSequenceTable(byte[] tableName) {
                     || Bytes.compareTo(tableName, SchemaUtil
                             .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES, true).getName()) == 0;
         }
    +
    +    public static boolean isTaskTable(byte[] tableName) {
    --- End diff --
    
    Checkout `StatisticsUtil` class. This is where we disable it.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223886613
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    I don't think we need a synchronous drop table. Once the parent is dropped any child views when they are later resolved will throw a TableNotFoundException as the parent doesn't exist. 


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223783433
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -2516,6 +2526,8 @@ public void dropTable(RpcController controller, DropTableRequest request,
                         }
                         throw new IllegalStateException(msg);
                     }
    +
    --- End diff --
    
    Done


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223783369
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java ---
    @@ -503,7 +503,10 @@ public void testTableMetadataScan() throws Exception {
                 assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.TYPE_SEQUENCE, PTableType.SYSTEM);
                 assertTrue(rs.next());
                 assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE, PTableType.SYSTEM);
    +            assertTrue(rs.next());
    +            assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_TASK_TABLE, PTableType.SYSTEM);
                 assertFalse(rs.next());
    +
    --- End diff --
    
    Done


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223897941
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -357,18 +357,28 @@
     			+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
     	
    -	 public static final String CREATE_MUTEX_METADTA =
    -	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    -	            // Pk columns
    -	            TENANT_ID + " VARCHAR NULL," +
    -	            TABLE_SCHEM + " VARCHAR NULL," +
    -	            TABLE_NAME + " VARCHAR NOT NULL," +
    -	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    -	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    -	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    -	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    -	            HConstants.VERSIONS + "=%s,\n" +
    -	            ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    -	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    -    
    +	public static final String CREATE_MUTEX_METADTA =
    --- End diff --
    
    sure, there is some indentation difference between 364-365, would be good to align them as well.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222448206
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java ---
    @@ -540,6 +540,12 @@ public static boolean isSequenceTable(byte[] tableName) {
                     || Bytes.compareTo(tableName, SchemaUtil
                             .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES, true).getName()) == 0;
         }
    +
    +    public static boolean isTaskTable(byte[] tableName) {
    --- End diff --
    
    There are also some other places in the code where we do special things for SYSTEM tables, one of the examples is that we disable stats for SYSTEM tables. It would good if you can make that pass to make things consistent across code. 


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222439605
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    --- End diff --
    
    You can instead have a protected variable that you can modify in tests. Adding a config prop is probably overkill.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by BinShi-SecularBird <gi...@git.apache.org>.
Github user BinShi-SecularBird commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223843964
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    From line 122 to the end of the test, the design of the test case is based on some assumptions which aren't rigorous and cause non-deterministic behavior. For example, the assertion at line 132 could fail at some extreme case because of resource contention on the test machine. I saw many test cases designed in this way were error prone and caused trouble eventually.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223888132
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, String tenantId, String schemaName, String tableName)
    +            throws IOException {
    +        try {
    --- End diff --
    
    Nice catch, if access checks are enabled the user won't have write access to the SYSTEM.TASK table, so you will need to upsert the row using User.runAsLoginUser (see MetaDataEndpointImpl.mutateRowsWithLocks).


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r224976070
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -357,18 +357,28 @@
     			+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
     	
    -	 public static final String CREATE_MUTEX_METADTA =
    -	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    -	            // Pk columns
    -	            TENANT_ID + " VARCHAR NULL," +
    -	            TABLE_SCHEM + " VARCHAR NULL," +
    -	            TABLE_NAME + " VARCHAR NOT NULL," +
    -	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    -	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    -	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    -	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    -	            HConstants.VERSIONS + "=%s,\n" +
    -	            ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    -	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    -    
    +	public static final String CREATE_MUTEX_METADTA =
    --- End diff --
    
    Will do


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222438781
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, String tenantId, String schemaName, String tableName)
    +            throws IOException {
    +        try {
    --- End diff --
    
    @twdsilva Do we now require WRITE access to users for doing this?
    Should we run this piece of code as `LoginUser` instead?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223780223
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, String tenantId, String schemaName, String tableName)
    +            throws IOException {
    +        try {
    --- End diff --
    
    @karanmehta93 and @twdsilva, please let me know if this is needed


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r221773742
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.cache.GlobalCache;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +        GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().invalidateAll();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        // sleep a little bit to compensate time clock skew when SYSTEM.CATALOG moves
    +        // among region servers because we relies on server time of RS which is hosting
    +        // SYSTEM.CATALOG
    +        Configuration config = env.getConfiguration();
    +        long sleepTime = config.getLong(QueryServices.CLOCK_SKEW_INTERVAL_ATTRIB,
    +            QueryServicesOptions.DEFAULT_CLOCK_SKEW_INTERVAL);
    +        try {
    +            if(sleepTime > 0) {
    +                Thread.sleep(sleepTime);
    +            }
    +        } catch (InterruptedException ie) {
    +            Thread.currentThread().interrupt();
    +        }
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, Timestamp ts, String tenantId, String schemaName, String tableName) throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME + " VALUES(?,?,?,?,?)");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            stmt.setTimestamp(2, ts);
    +            if (tenantId != null) {
    +                stmt.setString(3, tenantId);
    +            }else {
    +                stmt.setNull(3, Types.VARCHAR);
    +            }
    +            stmt.setString(4, schemaName);
    +            stmt.setString(5, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    public static void deleteTask(PhoenixConnection conn, TaskType taskType, Timestamp ts, String tenantId, String schemaName, String tableName) throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("DELETE FROM " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                    " WHERE " + PhoenixDatabaseMetaData.TASK_TYPE + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TASK_TS + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TENANT_ID + (tenantId == null ? " IS NULL " : " = '" + tenantId + "'") + " AND " +
    +                    PhoenixDatabaseMetaData.TABLE_SCHEM + "= ? AND " +
    +                    PhoenixDatabaseMetaData.TABLE_NAME + " = ?");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            stmt.setTimestamp(2, ts);
    +            stmt.setString(3, schemaName);
    +            stmt.setString(4, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    /**
    +     * Task runs periodically to clean up task of child views whose parent is dropped
    +     *
    +     */
    +    public static class DropChildViewsTask extends TimerTask {
    +        private RegionCoprocessorEnvironment env;
    +        private long timeMaxInterval;
    +
    +        public DropChildViewsTask(RegionCoprocessorEnvironment env, long timeMaxInterval) {
    +            this.env = env;
    +            this.timeMaxInterval = timeMaxInterval;
    +        }
    +  
    +        @Override
    +        public void run() {
    +            PhoenixConnection connForTask = null;
    +            try {
    +                String taskQuery = "SELECT " +
    +                        PhoenixDatabaseMetaData.TASK_TS + ", " +
    +                        PhoenixDatabaseMetaData.TENANT_ID + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_SCHEM + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_NAME +
    +                        " FROM " + PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                        " WHERE "+ PhoenixDatabaseMetaData.TASK_TYPE + " = " + PTable.TaskType.DROP_CHILD_VIEWS.getSerializedValue();
    +
    +                connForTask = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                PreparedStatement taskStatement = connForTask.prepareStatement(taskQuery);
    +                ResultSet rs = taskStatement.executeQuery();
    +                while (rs.next()) {
    +                    try {
    +                        // delete child views only if the parent table is deleted from the system catalog
    +                        Properties tenantProps = new Properties();
    +                        if (rs.getString(3) != null) {
    +                            tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, rs.getString(3));
    +                        }
    +                        PhoenixConnection pconn = QueryUtil.getConnectionOnServer(tenantProps, env.getConfiguration()).unwrap(PhoenixConnection.class);
    +
    +                        MetaDataProtocol.MetaDataMutationResult result = new MetaDataClient(pconn).updateCache(pconn.getTenantId(),
    +                                rs.getString(3), rs.getString(4), true);
    +                        if (result.getMutationCode() != MetaDataProtocol.MutationCode.TABLE_ALREADY_EXISTS) {
    +                            MetaDataEndpointImpl.dropChildViews(env, rs.getBytes(2), rs.getBytes(3), rs.getBytes(4));
    +                        } else if (System.currentTimeMillis() < timeMaxInterval + rs.getTimestamp(1).getTime()) {
    +                            // skip this task as it has not been expired and its parent table has not been dropped yet
    --- End diff --
    
    Maybe add an info log line here for this


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223886374
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    Instead of waiting we could just run the process that reads the tasks so that we don't have to rely on sleeping.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r226393542
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    See MutableIndexFailureIT.testIndexWriteFailure which just runs the BuildIndexScheduleTask


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r225707779
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    --- End diff --
    
    @karanmehta93, Ok, I will remove the configuration parameter for the initial delay


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r228266727
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java ---
    @@ -322,6 +322,12 @@
     
     	public static final String SYSTEM_CATALOG_SPLITTABLE = "phoenix.system.catalog.splittable";
     
    +    // The parameters defined for handling task stored in table SYSTEM.TASK
    +	// The time interval between periodic scans of table SYSTEM.TASK
    +    public static final String TASK_HANDLING_INTERVAL_ATTRIB = "phoenix.task.handling.interval";
    --- End diff --
    
    One small nit: Can we add timeunits in the property name?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r221773426
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.cache.GlobalCache;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +        GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().invalidateAll();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        // sleep a little bit to compensate time clock skew when SYSTEM.CATALOG moves
    +        // among region servers because we relies on server time of RS which is hosting
    +        // SYSTEM.CATALOG
    +        Configuration config = env.getConfiguration();
    +        long sleepTime = config.getLong(QueryServices.CLOCK_SKEW_INTERVAL_ATTRIB,
    +            QueryServicesOptions.DEFAULT_CLOCK_SKEW_INTERVAL);
    +        try {
    +            if(sleepTime > 0) {
    +                Thread.sleep(sleepTime);
    +            }
    +        } catch (InterruptedException ie) {
    +            Thread.currentThread().interrupt();
    +        }
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, Timestamp ts, String tenantId, String schemaName, String tableName) throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME + " VALUES(?,?,?,?,?)");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            stmt.setTimestamp(2, ts);
    +            if (tenantId != null) {
    +                stmt.setString(3, tenantId);
    +            }else {
    +                stmt.setNull(3, Types.VARCHAR);
    +            }
    +            stmt.setString(4, schemaName);
    +            stmt.setString(5, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    public static void deleteTask(PhoenixConnection conn, TaskType taskType, Timestamp ts, String tenantId, String schemaName, String tableName) throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("DELETE FROM " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                    " WHERE " + PhoenixDatabaseMetaData.TASK_TYPE + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TASK_TS + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TENANT_ID + (tenantId == null ? " IS NULL " : " = '" + tenantId + "'") + " AND " +
    +                    PhoenixDatabaseMetaData.TABLE_SCHEM + "= ? AND " +
    +                    PhoenixDatabaseMetaData.TABLE_NAME + " = ?");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            stmt.setTimestamp(2, ts);
    +            stmt.setString(3, schemaName);
    +            stmt.setString(4, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    /**
    +     * Task runs periodically to clean up task of child views whose parent is dropped
    +     *
    +     */
    +    public static class DropChildViewsTask extends TimerTask {
    +        private RegionCoprocessorEnvironment env;
    +        private long timeMaxInterval;
    +
    +        public DropChildViewsTask(RegionCoprocessorEnvironment env, long timeMaxInterval) {
    +            this.env = env;
    +            this.timeMaxInterval = timeMaxInterval;
    +        }
    +  
    +        @Override
    +        public void run() {
    +            PhoenixConnection connForTask = null;
    +            try {
    +                String taskQuery = "SELECT " +
    +                        PhoenixDatabaseMetaData.TASK_TS + ", " +
    +                        PhoenixDatabaseMetaData.TENANT_ID + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_SCHEM + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_NAME +
    +                        " FROM " + PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                        " WHERE "+ PhoenixDatabaseMetaData.TASK_TYPE + " = " + PTable.TaskType.DROP_CHILD_VIEWS.getSerializedValue();
    +
    +                connForTask = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                PreparedStatement taskStatement = connForTask.prepareStatement(taskQuery);
    +                ResultSet rs = taskStatement.executeQuery();
    +                while (rs.next()) {
    +                    try {
    +                        // delete child views only if the parent table is deleted from the system catalog
    +                        Properties tenantProps = new Properties();
    +                        if (rs.getString(3) != null) {
    --- End diff --
    
    the code is more readable if you assign this to a variable (rs.getString(..))


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r225701877
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    --- End diff --
    
    > I am not sure how you conclude that initialDelay is/will be only used for only test purposes.
    
    This is something that's only applicable for once during the lifetime of a region server, since phoenix lies inside HBase process. I am not sure if this will help. This is not the delay that will be applied before every task, is that correct? Also, I see that initialDelay is commented as "for test purposes" on Line 72 in TaskRegionObserver.java. 


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223888793
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -357,18 +357,28 @@
     			+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
     	
    -	 public static final String CREATE_MUTEX_METADTA =
    -	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    -	            // Pk columns
    -	            TENANT_ID + " VARCHAR NULL," +
    -	            TABLE_SCHEM + " VARCHAR NULL," +
    -	            TABLE_NAME + " VARCHAR NOT NULL," +
    -	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    -	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    -	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    -	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    -	            HConstants.VERSIONS + "=%s,\n" +
    -	            ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    -	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    -    
    +	public static final String CREATE_MUTEX_METADTA =
    +            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +             // Pk columns
    +            TENANT_ID + " VARCHAR NULL," +
    +            TABLE_SCHEM + " VARCHAR NULL," +
    +	        TABLE_NAME + " VARCHAR NOT NULL," +
    +	        COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	        COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	        "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	        + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	        HConstants.VERSIONS + "=%s,\n" +
    +	        ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    +	        PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    +
    +	public static final String CREATE_TASK_METADATA = "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\""
    +            + SYSTEM_TASK_TABLE + "\"(\n" +
    +            // PK columns
    +            TASK_TYPE + " UNSIGNED_TINYINT NOT NULL," + TASK_TS + " TIMESTAMP NOT NULL," + TENANT_ID + " VARCHAR NULL," + TABLE_SCHEM + " VARCHAR NULL," +
    +            TABLE_NAME + " VARCHAR NOT NULL,\n" +
    +            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TASK_TYPE + "," + TASK_TS + " ROW_TIMESTAMP," + TENANT_ID + "," + TABLE_SCHEM + "," +
    +            TABLE_NAME + "))\n" +
    +            HConstants.VERSIONS + "=%s,\n" +
    --- End diff --
    
    Its configurable with the default set 1.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223788723
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -357,18 +357,28 @@
     			+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
     	
    -	 public static final String CREATE_MUTEX_METADTA =
    -	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    -	            // Pk columns
    -	            TENANT_ID + " VARCHAR NULL," +
    -	            TABLE_SCHEM + " VARCHAR NULL," +
    -	            TABLE_NAME + " VARCHAR NOT NULL," +
    -	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    -	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    -	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    -	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    -	            HConstants.VERSIONS + "=%s,\n" +
    -	            ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    -	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    -    
    +	public static final String CREATE_MUTEX_METADTA =
    +            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +             // Pk columns
    +            TENANT_ID + " VARCHAR NULL," +
    +            TABLE_SCHEM + " VARCHAR NULL," +
    +	        TABLE_NAME + " VARCHAR NOT NULL," +
    +	        COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	        COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	        "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	        + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	        HConstants.VERSIONS + "=%s,\n" +
    +	        ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    +	        PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    +
    +	public static final String CREATE_TASK_METADATA = "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\""
    +            + SYSTEM_TASK_TABLE + "\"(\n" +
    +            // PK columns
    +            TASK_TYPE + " UNSIGNED_TINYINT NOT NULL," + TASK_TS + " TIMESTAMP NOT NULL," + TENANT_ID + " VARCHAR NULL," + TABLE_SCHEM + " VARCHAR NULL," +
    +            TABLE_NAME + " VARCHAR NOT NULL,\n" +
    +            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TASK_TYPE + "," + TASK_TS + " ROW_TIMESTAMP," + TENANT_ID + "," + TABLE_SCHEM + "," +
    +            TABLE_NAME + "))\n" +
    +            HConstants.VERSIONS + "=%s,\n" +
    --- End diff --
    
    Not sure about it. I am following the other system table schemas.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223783281
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -357,18 +357,28 @@
     			+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
     	
    -	 public static final String CREATE_MUTEX_METADTA =
    -	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    -	            // Pk columns
    -	            TENANT_ID + " VARCHAR NULL," +
    -	            TABLE_SCHEM + " VARCHAR NULL," +
    -	            TABLE_NAME + " VARCHAR NOT NULL," +
    -	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    -	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    -	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    -	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    -	            HConstants.VERSIONS + "=%s,\n" +
    -	            ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    -	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    -    
    +	public static final String CREATE_MUTEX_METADTA =
    +            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +             // Pk columns
    +            TENANT_ID + " VARCHAR NULL," +
    +            TABLE_SCHEM + " VARCHAR NULL," +
    +	        TABLE_NAME + " VARCHAR NOT NULL," +
    +	        COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	        COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	        "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	        + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	        HConstants.VERSIONS + "=%s,\n" +
    +	        ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    +	        PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    +
    +	public static final String CREATE_TASK_METADATA = "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\""
    +            + SYSTEM_TASK_TABLE + "\"(\n" +
    +            // PK columns
    +            TASK_TYPE + " UNSIGNED_TINYINT NOT NULL," + TASK_TS + " TIMESTAMP NOT NULL," + TENANT_ID + " VARCHAR NULL," + TABLE_SCHEM + " VARCHAR NULL," +
    --- End diff --
    
    Done


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223796635
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java ---
    @@ -540,6 +540,12 @@ public static boolean isSequenceTable(byte[] tableName) {
                     || Bytes.compareTo(tableName, SchemaUtil
                             .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES, true).getName()) == 0;
         }
    +
    +    public static boolean isTaskTable(byte[] tableName) {
    --- End diff --
    
    This comment is not specific and not actionable for me. I am not sure why we disable stats for system tables and where we do it.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r221769449
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,164 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    +
    +        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
    +        try (Connection conn = DriverManager.getConnection(getUrl());
    +                Connection viewConn =
    +                        isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn) {
    +            String ddlFormat =
    +                    "CREATE TABLE IF NOT EXISTS " + baseTable + "  ("
    +                            + " %s PK2 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR "
    +                            + " CONSTRAINT NAME_PK PRIMARY KEY (%s PK2)" + " ) %s";
    +            conn.createStatement().execute(generateDDL(ddlFormat));
    +            conn.commit();
    +
    +            // Create a view tree (i.e., tree of views) with depth of 2 and fanout factor of 4
    +            for (int  i = 0; i < 4; i++) {
    +                String childView = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
    +                String childViewDDL = "CREATE VIEW " + childView + " AS SELECT * FROM " + baseTable;
    +                viewConn.createStatement().execute(childViewDDL);
    +                for (int j = 0; j < 4; j++) {
    +                    String grandChildView = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
    +                    String grandChildViewDDL = "CREATE VIEW " + grandChildView + " AS SELECT * FROM " + childView;
    +                    viewConn.createStatement().execute(grandChildViewDDL);
    +                }
    +            }
    +            viewConn.commit();
    +
    +            // Drop the base table
    +            String dropTable = String.format("DROP TABLE IF EXISTS %s CASCADE", baseTable);
    +            conn.createStatement().execute(dropTable);
    +            conn.commit();
    +
    +            try {
    +                PhoenixRuntime.getTableNoCache(viewConn, baseTable.toUpperCase());
    +                fail("An exception was not raised! the deleted table is loaded successfully");
    +            }
    +            catch (Throwable t) {
    --- End diff --
    
    catch the expected TableNotFoundException here (in case something else is thrown)


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by BinShi-SecularBird <gi...@git.apache.org>.
Github user BinShi-SecularBird commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223906186
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -2655,10 +2666,19 @@ private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] s
                         boolean hasChildViews =
                                 ViewFinder.hasChildViews(hTable, tenantId, schemaName, tableName,
                                     clientTimeStamp);
    -                    if (hasChildViews && !isCascade) {
    -                        // DROP without CASCADE on tables with child views is not permitted
    -                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
    -                                EnvironmentEdgeManager.currentTimeMillis(), null);
    +                    if (hasChildViews) {
    +                        if (!isCascade) {
    +                            // DROP without CASCADE on tables with child views is not permitted
    --- End diff --
    
    Why do we have this limitation? Shouldn't DROP mean "DROP CASCADE" in the case that the table has child views? This comment is orthogonal to this pull request as it exists in the original code base. 


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r221771281
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,164 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    +
    +        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
    +        try (Connection conn = DriverManager.getConnection(getUrl());
    +                Connection viewConn =
    +                        isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn) {
    +            String ddlFormat =
    +                    "CREATE TABLE IF NOT EXISTS " + baseTable + "  ("
    +                            + " %s PK2 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR "
    +                            + " CONSTRAINT NAME_PK PRIMARY KEY (%s PK2)" + " ) %s";
    +            conn.createStatement().execute(generateDDL(ddlFormat));
    +            conn.commit();
    +
    +            // Create a view tree (i.e., tree of views) with depth of 2 and fanout factor of 4
    +            for (int  i = 0; i < 4; i++) {
    +                String childView = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
    +                String childViewDDL = "CREATE VIEW " + childView + " AS SELECT * FROM " + baseTable;
    +                viewConn.createStatement().execute(childViewDDL);
    +                for (int j = 0; j < 4; j++) {
    +                    String grandChildView = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
    +                    String grandChildViewDDL = "CREATE VIEW " + grandChildView + " AS SELECT * FROM " + childView;
    +                    viewConn.createStatement().execute(grandChildViewDDL);
    +                }
    +            }
    +            viewConn.commit();
    +
    +            // Drop the base table
    +            String dropTable = String.format("DROP TABLE IF EXISTS %s CASCADE", baseTable);
    +            conn.createStatement().execute(dropTable);
    +            conn.commit();
    +
    +            try {
    +                PhoenixRuntime.getTableNoCache(viewConn, baseTable.toUpperCase());
    +                fail("An exception was not raised! the deleted table is loaded successfully");
    +            }
    +            catch (Throwable t) {
    +                // As expected a Throwable exception happened
    +            }
    +
    +            // Most of the views should be still available immediately after table drop
    +            TableViewFinderResult childViewsResultBefore = new TableViewFinderResult();
    +            TableName linkTable = TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES);
    +            ViewFinder.findAllRelatives(getUtility().getConnection().getTable(linkTable),
    +                    HConstants.EMPTY_BYTE_ARRAY,
    +                    SchemaUtil.getSchemaNameFromFullName(baseTable).getBytes(),
    +                    SchemaUtil.getTableNameFromFullName(baseTable).getBytes(),
    +                    PTable.LinkType.CHILD_TABLE,
    +                    childViewsResultBefore);
    +            assertTrue(childViewsResultBefore.getLinks().size() > 0);
    +
    +            // Wait for the tasks for dropping child views to complete. The depth of the view tree is 2, so we should wait
    +            // 3 times the task handling interval by assuming that each non-root level will be processed in one interval
    +
    +            long timeInterval = config.getLong(
    +                            QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                            QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +
    +            Thread.sleep(3*timeInterval);
    --- End diff --
    
    This might flap in some environments, can you run the task manually here so that the child views get cleaned up?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222443397
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, String tenantId, String schemaName, String tableName)
    +            throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME + " (" +
    +                    PhoenixDatabaseMetaData.TASK_TYPE + ", " +
    +                    PhoenixDatabaseMetaData.TENANT_ID + ", " +
    +                    PhoenixDatabaseMetaData.TABLE_SCHEM + ", " +
    +                    PhoenixDatabaseMetaData.TABLE_NAME + ") VALUES(?,?,?,?)");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            if (tenantId != null) {
    +                stmt.setString(2, tenantId);
    +            }else {
    --- End diff --
    
    nit: spaces before/after brackets.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222444240
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -357,18 +357,28 @@
     			+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
     	
    -	 public static final String CREATE_MUTEX_METADTA =
    -	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    -	            // Pk columns
    -	            TENANT_ID + " VARCHAR NULL," +
    -	            TABLE_SCHEM + " VARCHAR NULL," +
    -	            TABLE_NAME + " VARCHAR NOT NULL," +
    -	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    -	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    -	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    -	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    -	            HConstants.VERSIONS + "=%s,\n" +
    -	            ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    -	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    -    
    +	public static final String CREATE_MUTEX_METADTA =
    --- End diff --
    
    nit: unnecessary diff


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223786436
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -357,18 +357,28 @@
     			+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
     	
    -	 public static final String CREATE_MUTEX_METADTA =
    -	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    -	            // Pk columns
    -	            TENANT_ID + " VARCHAR NULL," +
    -	            TABLE_SCHEM + " VARCHAR NULL," +
    -	            TABLE_NAME + " VARCHAR NOT NULL," +
    -	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    -	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    -	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    -	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    -	            HConstants.VERSIONS + "=%s,\n" +
    -	            ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    -	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    -    
    +	public static final String CREATE_MUTEX_METADTA =
    --- End diff --
    
    Yes, but was annoying to leave it with incorrect indentation.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223897197
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    --- End diff --
    
    We dont need this thing anymore since Java 1.5+, should be okay to remove.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223779808
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    --- End diff --
    
    Yes, we should do it when we support more than one task type and a specific task needs more than one thread. Currently, DROP_CHILD_VIEW task needs only thread. I did not want to complicate it unnecessarily.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r221773219
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.cache.GlobalCache;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +        GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().invalidateAll();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        // sleep a little bit to compensate time clock skew when SYSTEM.CATALOG moves
    +        // among region servers because we relies on server time of RS which is hosting
    +        // SYSTEM.CATALOG
    +        Configuration config = env.getConfiguration();
    +        long sleepTime = config.getLong(QueryServices.CLOCK_SKEW_INTERVAL_ATTRIB,
    +            QueryServicesOptions.DEFAULT_CLOCK_SKEW_INTERVAL);
    +        try {
    +            if(sleepTime > 0) {
    +                Thread.sleep(sleepTime);
    +            }
    +        } catch (InterruptedException ie) {
    +            Thread.currentThread().interrupt();
    +        }
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, Timestamp ts, String tenantId, String schemaName, String tableName) throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME + " VALUES(?,?,?,?,?)");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            stmt.setTimestamp(2, ts);
    +            if (tenantId != null) {
    +                stmt.setString(3, tenantId);
    +            }else {
    +                stmt.setNull(3, Types.VARCHAR);
    +            }
    +            stmt.setString(4, schemaName);
    +            stmt.setString(5, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    public static void deleteTask(PhoenixConnection conn, TaskType taskType, Timestamp ts, String tenantId, String schemaName, String tableName) throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("DELETE FROM " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                    " WHERE " + PhoenixDatabaseMetaData.TASK_TYPE + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TASK_TS + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TENANT_ID + (tenantId == null ? " IS NULL " : " = '" + tenantId + "'") + " AND " +
    +                    PhoenixDatabaseMetaData.TABLE_SCHEM + "= ? AND " +
    +                    PhoenixDatabaseMetaData.TABLE_NAME + " = ?");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            stmt.setTimestamp(2, ts);
    +            stmt.setString(3, schemaName);
    +            stmt.setString(4, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    /**
    +     * Task runs periodically to clean up task of child views whose parent is dropped
    +     *
    +     */
    +    public static class DropChildViewsTask extends TimerTask {
    +        private RegionCoprocessorEnvironment env;
    +        private long timeMaxInterval;
    +
    +        public DropChildViewsTask(RegionCoprocessorEnvironment env, long timeMaxInterval) {
    +            this.env = env;
    +            this.timeMaxInterval = timeMaxInterval;
    +        }
    +  
    +        @Override
    +        public void run() {
    +            PhoenixConnection connForTask = null;
    +            try {
    +                String taskQuery = "SELECT " +
    +                        PhoenixDatabaseMetaData.TASK_TS + ", " +
    +                        PhoenixDatabaseMetaData.TENANT_ID + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_SCHEM + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_NAME +
    +                        " FROM " + PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                        " WHERE "+ PhoenixDatabaseMetaData.TASK_TYPE + " = " + PTable.TaskType.DROP_CHILD_VIEWS.getSerializedValue();
    +
    +                connForTask = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                PreparedStatement taskStatement = connForTask.prepareStatement(taskQuery);
    +                ResultSet rs = taskStatement.executeQuery();
    +                while (rs.next()) {
    +                    try {
    +                        // delete child views only if the parent table is deleted from the system catalog
    +                        Properties tenantProps = new Properties();
    +                        if (rs.getString(3) != null) {
    +                            tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, rs.getString(3));
    +                        }
    +                        PhoenixConnection pconn = QueryUtil.getConnectionOnServer(tenantProps, env.getConfiguration()).unwrap(PhoenixConnection.class);
    +
    +                        MetaDataProtocol.MetaDataMutationResult result = new MetaDataClient(pconn).updateCache(pconn.getTenantId(),
    +                                rs.getString(3), rs.getString(4), true);
    +                        if (result.getMutationCode() != MetaDataProtocol.MutationCode.TABLE_ALREADY_EXISTS) {
    +                            MetaDataEndpointImpl.dropChildViews(env, rs.getBytes(2), rs.getBytes(3), rs.getBytes(4));
    +                        } else if (System.currentTimeMillis() < timeMaxInterval + rs.getTimestamp(1).getTime()) {
    +                            // skip this task as it has not been expired and its parent table has not been dropped yet
    +                            continue;
    +                        }
    +
    +                        deleteTask(connForTask, PTable.TaskType.DROP_CHILD_VIEWS, rs.getTimestamp(1), rs.getString(2),
    +                                rs.getString(3), rs.getString(4));
    +                    }
    +                    catch (Throwable t) {
    +                        LOG.warn("Exception while deleting child view " + rs.getString(3) + "." + rs.getString(4) +
    +                                " with tenant id " + (rs.getString(2) == null ? " IS NULL" : rs.getString(2)) +
    +                                " and timestamp " + rs.getTimestamp(1).toString(), t);
    +                    }
    +                }
    +            } catch (Throwable t) {
    +                LOG.warn("DropChildViewsTask failed!", t);
    --- End diff --
    
    Log this as an error


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r224976568
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    --- End diff --
    
    @karanmehta93, I am new to the Phoenix code and following the styles in the existing code base. MetaDataObserver has these parameters defined for the same purposes. I can imagine that we may need to change these values if we hit some issues and a single default value may not be sufficient. I am not sure how you conclude that initialDelay is/will be only used for only test purposes. 


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223897107
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    --- End diff --
    
    We can actually simplify by not exposing these properties out. The defaults seem to be good enough.
    Especially the `initialDelay` parameter which is only used for test purposes.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r224975238
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    --- End diff --
    
    Will do


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r224975650
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    I tried and really it did not add any value and just added more code. The methods includes 80 lines, and 10 of them are comments and 8 of them are "}".  There are two method calls take 12 lines because they have 6 arguments and each argument is written on a separate line. I do not see what bothers you here. You can break into tiny methods, such as createBaseTable, createViews, dropBaseTable,  and waitForViewsToDrop, if these methods are used by other test methods. At this moment, there is only one test method.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by BinShi-SecularBird <gi...@git.apache.org>.
Github user BinShi-SecularBird commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223835836
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    --- End diff --
    
    Add white space before and after !=?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r221772088
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -2516,6 +2526,8 @@ public void dropTable(RpcController controller, DropTableRequest request,
                         }
                         throw new IllegalStateException(msg);
                     }
    +
    --- End diff --
    
    nit: whitespace


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222444065
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java ---
    @@ -3206,6 +3219,9 @@ public void upgradeSystemTables(final String url, final Properties props) throws
                 try {
                     metaConnection.createStatement().executeUpdate(getMutexDDL());
                 } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
    +            try {
    --- End diff --
    
    nit: Can we refactor and extract out these dupe pieces of code into a method?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222444508
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -357,18 +357,28 @@
     			+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
     	
    -	 public static final String CREATE_MUTEX_METADTA =
    -	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    -	            // Pk columns
    -	            TENANT_ID + " VARCHAR NULL," +
    -	            TABLE_SCHEM + " VARCHAR NULL," +
    -	            TABLE_NAME + " VARCHAR NOT NULL," +
    -	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    -	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    -	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    -	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    -	            HConstants.VERSIONS + "=%s,\n" +
    -	            ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    -	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    -    
    +	public static final String CREATE_MUTEX_METADTA =
    +            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +             // Pk columns
    +            TENANT_ID + " VARCHAR NULL," +
    +            TABLE_SCHEM + " VARCHAR NULL," +
    +	        TABLE_NAME + " VARCHAR NOT NULL," +
    +	        COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	        COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	        "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	        + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	        HConstants.VERSIONS + "=%s,\n" +
    +	        ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    +	        PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    +
    +	public static final String CREATE_TASK_METADATA = "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\""
    +            + SYSTEM_TASK_TABLE + "\"(\n" +
    +            // PK columns
    +            TASK_TYPE + " UNSIGNED_TINYINT NOT NULL," + TASK_TS + " TIMESTAMP NOT NULL," + TENANT_ID + " VARCHAR NULL," + TABLE_SCHEM + " VARCHAR NULL," +
    --- End diff --
    
    nit: different column names in different lines for better readability. 


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223898108
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java ---
    @@ -192,7 +192,39 @@ public static LinkType fromSerializedValue(byte serializedValue) {
                 return LinkType.values()[serializedValue-1];
             }
         }
    -    
    +
    +    public enum TaskType {
    +        DROP_CHILD_VIEWS((byte)1);
    +
    +        private final byte[] byteValue;
    +        private final byte serializedValue;
    +
    +        TaskType(byte serializedValue) {
    +            this.serializedValue = serializedValue;
    +            this.byteValue = Bytes.toBytes(this.name());
    +        }
    +
    +        public byte[] getBytes() {
    +                return byteValue;
    +        }
    +
    +        public byte getSerializedValue() {
    +            return this.serializedValue;
    +        }
    +        public static TaskType getDefault() {
    +            return DROP_CHILD_VIEWS;
    +        }
    +        public static TaskType fromToken(String token) {
    +            return TaskType.valueOf(token.trim().toUpperCase());
    +        }
    +        public static TaskType fromSerializedValue(byte serializedValue) {
    --- End diff --
    
    Looking at the code in IDE, I dont see it being used anywhere.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r224976392
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java ---
    @@ -540,6 +540,12 @@ public static boolean isSequenceTable(byte[] tableName) {
                     || Bytes.compareTo(tableName, SchemaUtil
                             .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES, true).getName()) == 0;
         }
    +
    +    public static boolean isTaskTable(byte[] tableName) {
    --- End diff --
    
    Will do


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222445242
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -357,18 +357,28 @@
     			+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
     	
    -	 public static final String CREATE_MUTEX_METADTA =
    -	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    -	            // Pk columns
    -	            TENANT_ID + " VARCHAR NULL," +
    -	            TABLE_SCHEM + " VARCHAR NULL," +
    -	            TABLE_NAME + " VARCHAR NOT NULL," +
    -	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    -	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    -	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    -	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    -	            HConstants.VERSIONS + "=%s,\n" +
    -	            ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    -	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    -    
    +	public static final String CREATE_MUTEX_METADTA =
    +            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +             // Pk columns
    +            TENANT_ID + " VARCHAR NULL," +
    +            TABLE_SCHEM + " VARCHAR NULL," +
    +	        TABLE_NAME + " VARCHAR NOT NULL," +
    +	        COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	        COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	        "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	        + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	        HConstants.VERSIONS + "=%s,\n" +
    +	        ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    +	        PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    +
    +	public static final String CREATE_TASK_METADATA = "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\""
    +            + SYSTEM_TASK_TABLE + "\"(\n" +
    +            // PK columns
    +            TASK_TYPE + " UNSIGNED_TINYINT NOT NULL," + TASK_TS + " TIMESTAMP NOT NULL," + TENANT_ID + " VARCHAR NULL," + TABLE_SCHEM + " VARCHAR NULL," +
    +            TABLE_NAME + " VARCHAR NOT NULL,\n" +
    +            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TASK_TYPE + "," + TASK_TS + " ROW_TIMESTAMP," + TENANT_ID + "," + TABLE_SCHEM + "," +
    +            TABLE_NAME + "))\n" +
    +            HConstants.VERSIONS + "=%s,\n" +
    --- End diff --
    
    What value will be populated here? Can we fix the `MAX_VERSIONS` to 1 to make things simpler?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r224975964
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    @BinShi-SecularBird and @twdsilva, I do not see how you can eliminate sleeping here as long as we need to poll a table (i.e., read the task table, system catalog table, or link table) because tasks are executed asynchronously. Please note that in order to eliminate unnecessary sleeping, the sleep time is set to half of the interval time (i.e., delay) between subsequent executions of the task table thread. It is expected that in tasks will be executed in 2 thread run, i.e., at most in three intervals. Due to "non-deterministic" nature of thread scheduling, the test waits 10 intervals, which should be sufficient almost always, I think. If you think we need wait more, please let me know it.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by BinShi-SecularBird <gi...@git.apache.org>.
Github user BinShi-SecularBird commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223905380
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---
    @@ -2346,19 +2347,29 @@ public void createTable(RpcController controller, CreateTableRequest request,
             }
         }
     
    -    private void dropChildMetadata(byte[] schemaName, byte[] tableName, byte[] tenantIdBytes)
    +    public static void dropChildViews(RegionCoprocessorEnvironment env, byte[] tenantIdBytes, byte[] schemaName, byte[] tableName)
                 throws IOException, SQLException, ClassNotFoundException {
    -        TableViewFinderResult childViewsResult = new TableViewFinderResult();
    -        findAllChildViews(tenantIdBytes, schemaName, tableName, childViewsResult);
    +        Table hTable =
    +                ServerUtil.getHTableForCoprocessorScan(env,
    +                        SchemaUtil.getPhysicalTableName(
    +                                PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
    +                                env.getConfiguration()));
    +        TableViewFinderResult childViewsResult = ViewFinder.findRelatedViews(hTable, tenantIdBytes, schemaName, tableName,
    +                PTable.LinkType.CHILD_TABLE, HConstants.LATEST_TIMESTAMP);
    +
             if (childViewsResult.hasLinks()) {
    +
                 for (TableInfo viewInfo : childViewsResult.getLinks()) {
                     byte[] viewTenantId = viewInfo.getTenantId();
                     byte[] viewSchemaName = viewInfo.getSchemaName();
                     byte[] viewName = viewInfo.getTableName();
    +                logger.debug("dropChildViews :" + Bytes.toString(schemaName) + "." + Bytes.toString(tableName) +
    +                        " -> " + Bytes.toString(viewSchemaName) + "." + Bytes.toString(viewName) +
    +                        "with tenant id :" + Bytes.toString(viewTenantId));
    --- End diff --
    
    In the code base, even just in this file, I saw it prints debug log only when logger.isDebugEnabled() is true. The only exception is that few places print debug info for rare case or failure case (actually it should be changed to info or error log). You might want to add if (logger.isDebugEnabled()) { ... } too for printing debug log.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223897753
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java ---
    @@ -3206,6 +3219,9 @@ public void upgradeSystemTables(final String url, final Properties props) throws
                 try {
                     metaConnection.createStatement().executeUpdate(getMutexDDL());
                 } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
    +            try {
    --- End diff --
    
    Sorry for not being clear, we can refactor the method `upgradeSystemTables` and `createOtherSystemTables` in this one.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r228005500
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.cache.GlobalCache;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +        GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().invalidateAll();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        // sleep a little bit to compensate time clock skew when SYSTEM.CATALOG moves
    +        // among region servers because we relies on server time of RS which is hosting
    +        // SYSTEM.CATALOG
    +        Configuration config = env.getConfiguration();
    --- End diff --
    
    Done


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223779347
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    It is not really a big test when compared to others.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r224976381
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, String tenantId, String schemaName, String tableName)
    +            throws IOException {
    +        try {
    --- End diff --
    
    Will do


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r221772735
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.cache.GlobalCache;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +        GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().invalidateAll();
    --- End diff --
    
    I don't think we need to do this.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r224976390
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java ---
    @@ -192,7 +192,39 @@ public static LinkType fromSerializedValue(byte serializedValue) {
                 return LinkType.values()[serializedValue-1];
             }
         }
    -    
    +
    +    public enum TaskType {
    +        DROP_CHILD_VIEWS((byte)1);
    +
    +        private final byte[] byteValue;
    +        private final byte serializedValue;
    +
    +        TaskType(byte serializedValue) {
    +            this.serializedValue = serializedValue;
    +            this.byteValue = Bytes.toBytes(this.name());
    +        }
    +
    +        public byte[] getBytes() {
    +                return byteValue;
    +        }
    +
    +        public byte getSerializedValue() {
    +            return this.serializedValue;
    +        }
    +        public static TaskType getDefault() {
    +            return DROP_CHILD_VIEWS;
    +        }
    +        public static TaskType fromToken(String token) {
    +            return TaskType.valueOf(token.trim().toUpperCase());
    +        }
    +        public static TaskType fromSerializedValue(byte serializedValue) {
    --- End diff --
    
    It is used in my IDE


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222438220
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    --- End diff --
    
    Why do we have this? What is originating them?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223787638
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---
    @@ -357,18 +357,28 @@
     			+ ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
     			+ Boolean.FALSE;
     	
    -	 public static final String CREATE_MUTEX_METADTA =
    -	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    -	            // Pk columns
    -	            TENANT_ID + " VARCHAR NULL," +
    -	            TABLE_SCHEM + " VARCHAR NULL," +
    -	            TABLE_NAME + " VARCHAR NOT NULL," +
    -	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
    -	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    -	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    -	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    -	            HConstants.VERSIONS + "=%s,\n" +
    -	            ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    -	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    -    
    +	public static final String CREATE_MUTEX_METADTA =
    +            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
    +             // Pk columns
    +            TENANT_ID + " VARCHAR NULL," +
    +            TABLE_SCHEM + " VARCHAR NULL," +
    +	        TABLE_NAME + " VARCHAR NOT NULL," +
    +	        COLUMN_NAME + " VARCHAR NULL," + // null for table row
    +	        COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
    +	        "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
    +	        + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
    +	        HConstants.VERSIONS + "=%s,\n" +
    +	        ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS + "=%s,\n" +
    +	        PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
    +
    +	public static final String CREATE_TASK_METADATA = "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\""
    +            + SYSTEM_TASK_TABLE + "\"(\n" +
    +            // PK columns
    +            TASK_TYPE + " UNSIGNED_TINYINT NOT NULL," + TASK_TS + " TIMESTAMP NOT NULL," + TENANT_ID + " VARCHAR NULL," + TABLE_SCHEM + " VARCHAR NULL," +
    --- End diff --
    
    Done


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223779607
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    --- End diff --
    
    It is also in the MetaDataRegionObserver:
    /**
     * Coprocessor for metadata related operations. This coprocessor would only be registered
     * to SYSTEM.TABLE.
     */
    @SuppressWarnings("deprecation")
    public class MetaDataRegionObserver implements RegionObserver,RegionCoprocessor {
    
    I will try to remove it


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223780020
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    --- End diff --
    
    I will add comments to QueryServices.java file where these configuration parameters are defined


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by BinShi-SecularBird <gi...@git.apache.org>.
Github user BinShi-SecularBird commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223844408
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    Maybe you want to add synchronous implementation for "DROP TABLE IF EXISTS ... CASCADE" even just for test purpose, just like what we've done for "UPDATE STATISTICS"


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by BinShi-SecularBird <gi...@git.apache.org>.
Github user BinShi-SecularBird commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223835796
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    --- End diff --
    
    Add white space before and after !=?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223792014
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java ---
    @@ -192,7 +192,39 @@ public static LinkType fromSerializedValue(byte serializedValue) {
                 return LinkType.values()[serializedValue-1];
             }
         }
    -    
    +
    +    public enum TaskType {
    +        DROP_CHILD_VIEWS((byte)1);
    +
    +        private final byte[] byteValue;
    +        private final byte serializedValue;
    +
    +        TaskType(byte serializedValue) {
    +            this.serializedValue = serializedValue;
    +            this.byteValue = Bytes.toBytes(this.name());
    +        }
    +
    +        public byte[] getBytes() {
    +                return byteValue;
    +        }
    +
    +        public byte getSerializedValue() {
    +            return this.serializedValue;
    +        }
    +        public static TaskType getDefault() {
    +            return DROP_CHILD_VIEWS;
    +        }
    +        public static TaskType fromToken(String token) {
    +            return TaskType.valueOf(token.trim().toUpperCase());
    +        }
    +        public static TaskType fromSerializedValue(byte serializedValue) {
    --- End diff --
    
    @twdsilva, If it is not required and then we should remove it from both. 


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r224976378
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    --- End diff --
    
    Will do


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r221771351
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java ---
    @@ -503,7 +503,10 @@ public void testTableMetadataScan() throws Exception {
                 assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.TYPE_SEQUENCE, PTableType.SYSTEM);
                 assertTrue(rs.next());
                 assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE, PTableType.SYSTEM);
    +            assertTrue(rs.next());
    +            assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_TASK_TABLE, PTableType.SYSTEM);
                 assertFalse(rs.next());
    +
    --- End diff --
    
    nit: extra whitespace


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222443224
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.cache.GlobalCache;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +        GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().invalidateAll();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        // sleep a little bit to compensate time clock skew when SYSTEM.CATALOG moves
    +        // among region servers because we relies on server time of RS which is hosting
    +        // SYSTEM.CATALOG
    +        Configuration config = env.getConfiguration();
    --- End diff --
    
    I have seen checks around `CoprocessorEnvironment` instance type in various pieces of code. Do you think we need one here?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222440911
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    --- End diff --
    
    I think that its better to make the number of threads configurable and not based on `TaskType` only. We don't often drop tables, but if the variety and heterogeneity of tasks grow in future, we definitely should be able to get more. IMHO, 4 threads should be good default to start with.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222442226
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    --- End diff --
    
    nit: `TaskRegionObserver` --> `SystemTaskRegionObserver`?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by dbwong <gi...@git.apache.org>.
Github user dbwong commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223822500
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, String tenantId, String schemaName, String tableName)
    +            throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME + " ( " +
    +                    PhoenixDatabaseMetaData.TASK_TYPE + ", " +
    +                    PhoenixDatabaseMetaData.TENANT_ID + ", " +
    +                    PhoenixDatabaseMetaData.TABLE_SCHEM + ", " +
    +                    PhoenixDatabaseMetaData.TABLE_NAME + " ) VALUES(?,?,?,?)");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            if (tenantId != null) {
    +                stmt.setString(2, tenantId);
    +            }else {
    +                stmt.setNull(2, Types.VARCHAR);
    +            }
    +            stmt.setString(3, schemaName);
    +            stmt.setString(4, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    public static void deleteTask(PhoenixConnection conn, TaskType taskType, Timestamp ts, String tenantId, String schemaName, String tableName) throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("DELETE FROM " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                    " WHERE " + PhoenixDatabaseMetaData.TASK_TYPE + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TASK_TS + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TENANT_ID + (tenantId == null ? " IS NULL " : " = '" + tenantId + "'") + " AND " +
    +                    PhoenixDatabaseMetaData.TABLE_SCHEM + "= ? AND " +
    +                    PhoenixDatabaseMetaData.TABLE_NAME + " = ?");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            stmt.setTimestamp(2, ts);
    +            stmt.setString(3, schemaName);
    +            stmt.setString(4, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    /**
    +     * Task runs periodically to clean up task of child views whose parent is dropped
    +     *
    +     */
    +    public static class DropChildViewsTask extends TimerTask {
    +        private RegionCoprocessorEnvironment env;
    +        private long timeMaxInterval;
    +
    +        public DropChildViewsTask(RegionCoprocessorEnvironment env, long timeMaxInterval) {
    +            this.env = env;
    +            this.timeMaxInterval = timeMaxInterval;
    +        }
    +  
    +        @Override
    +        public void run() {
    +            PhoenixConnection connForTask = null;
    +            Timestamp timestamp = null;
    +            String tenantId = null;
    +            byte[] tenantIdBytes;
    +            String schemaName= null;
    +            byte[] schemaNameBytes;
    +            String tableName = null;
    +            byte[] tableNameBytes;
    +            PhoenixConnection pconn;
    +            try {
    +                String taskQuery = "SELECT " +
    +                        PhoenixDatabaseMetaData.TASK_TS + ", " +
    +                        PhoenixDatabaseMetaData.TENANT_ID + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_SCHEM + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_NAME +
    +                        " FROM " + PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                        " WHERE "+ PhoenixDatabaseMetaData.TASK_TYPE + " = " + PTable.TaskType.DROP_CHILD_VIEWS.getSerializedValue();
    +
    +                connForTask = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                PreparedStatement taskStatement = connForTask.prepareStatement(taskQuery);
    +                ResultSet rs = taskStatement.executeQuery();
    +                while (rs.next()) {
    +                    try {
    +                        // delete child views only if the parent table is deleted from the system catalog
    +                        timestamp = rs.getTimestamp(1);
    +                        tenantId = rs.getString(2);
    +                        tenantIdBytes= rs.getBytes(2);
    +                        schemaName= rs.getString(3);
    +                        schemaNameBytes = rs.getBytes(3);
    +                        tableName= rs.getString(4);
    +                        tableNameBytes = rs.getBytes(4);
    +
    +                        if (tenantId != null) {
    +                            Properties tenantProps = new Properties();
    +                            tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
    +                            pconn = QueryUtil.getConnectionOnServer(tenantProps, env.getConfiguration()).unwrap(PhoenixConnection.class);
    +
    +                        }
    +                        else {
    +                            pconn = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                        }
    +
    +                        MetaDataProtocol.MetaDataMutationResult result = new MetaDataClient(pconn).updateCache(pconn.getTenantId(),
    --- End diff --
    
    Why are we making a new MetaDataClient per result?  Can't we make it per pconn?


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223875740
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java ---
    @@ -3206,6 +3219,9 @@ public void upgradeSystemTables(final String url, final Properties props) throws
                 try {
                     metaConnection.createStatement().executeUpdate(getMutexDDL());
                 } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
    +            try {
    --- End diff --
    
    No sure about any value of extracting this simple try catch into a method.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r224976013
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java ---
    @@ -3206,6 +3219,9 @@ public void upgradeSystemTables(final String url, final Properties props) throws
                 try {
                     metaConnection.createStatement().executeUpdate(getMutexDDL());
                 } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
    +            try {
    --- End diff --
    
    @karanmehta93, there is really no duplicate pieces of code here. Note that these methods have different try-catch logic


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by twdsilva <gi...@git.apache.org>.
Github user twdsilva commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223886927
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.cache.GlobalCache;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +        GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().invalidateAll();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        // sleep a little bit to compensate time clock skew when SYSTEM.CATALOG moves
    +        // among region servers because we relies on server time of RS which is hosting
    +        // SYSTEM.CATALOG
    +        Configuration config = env.getConfiguration();
    --- End diff --
    
    I don't think its needed.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r226685573
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    That is not the right way of doing an integration testing. That would be more of a unit test. I should not call an internal function directly and verify that it works for the purpose of integration testing. This is one reason not to that. The second reason is that I wanted to verify that the coprocessor (TaskRegionObserver) is initialized,  it creates an instance of DropChildViewsTask,  and this instance is run at regular intervals in addition to DropChildViewsTask functions correctly. The third reason is that even I create another instance of DropChildViewsTask and call it directly, I would not know which instance processed the drop child tasks in the system task table. A view tree can have more one level of views and the run method of DropChildViewsTask needs to be called once for each level. This also introduces concurrency which is not supported by the current implementation.  Please note that I set the scheduling interval to 1 sec  (in BaseTest.java) for testing purpose. Sin
 ce the test sleeps only half this interval at a time, this test should be done in 2.5 seconds on average (for a view tree of depth 2). The sleeping logic here can only sleep extra 0.5 sec at most. Of course, if the threads are not scheduled on time, this test can take longer. For that reason, the test waits up to 10 seconds. 


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222442511
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    --- End diff --
    
    nit: good to have comment or doc here about all the props that we are making configurable.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222443657
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.cache.GlobalCache;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +        GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().invalidateAll();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        // sleep a little bit to compensate time clock skew when SYSTEM.CATALOG moves
    +        // among region servers because we relies on server time of RS which is hosting
    +        // SYSTEM.CATALOG
    +        Configuration config = env.getConfiguration();
    +        long sleepTime = config.getLong(QueryServices.CLOCK_SKEW_INTERVAL_ATTRIB,
    +            QueryServicesOptions.DEFAULT_CLOCK_SKEW_INTERVAL);
    +        try {
    +            if(sleepTime > 0) {
    +                Thread.sleep(sleepTime);
    +            }
    +        } catch (InterruptedException ie) {
    +            Thread.currentThread().interrupt();
    +        }
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, Timestamp ts, String tenantId, String schemaName, String tableName) throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME + " VALUES(?,?,?,?,?)");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            stmt.setTimestamp(2, ts);
    +            if (tenantId != null) {
    +                stmt.setString(3, tenantId);
    +            }else {
    +                stmt.setNull(3, Types.VARCHAR);
    +            }
    +            stmt.setString(4, schemaName);
    +            stmt.setString(5, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    public static void deleteTask(PhoenixConnection conn, TaskType taskType, Timestamp ts, String tenantId, String schemaName, String tableName) throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("DELETE FROM " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                    " WHERE " + PhoenixDatabaseMetaData.TASK_TYPE + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TASK_TS + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TENANT_ID + (tenantId == null ? " IS NULL " : " = '" + tenantId + "'") + " AND " +
    +                    PhoenixDatabaseMetaData.TABLE_SCHEM + "= ? AND " +
    +                    PhoenixDatabaseMetaData.TABLE_NAME + " = ?");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            stmt.setTimestamp(2, ts);
    +            stmt.setString(3, schemaName);
    +            stmt.setString(4, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    /**
    +     * Task runs periodically to clean up task of child views whose parent is dropped
    +     *
    +     */
    +    public static class DropChildViewsTask extends TimerTask {
    +        private RegionCoprocessorEnvironment env;
    +        private long timeMaxInterval;
    +
    +        public DropChildViewsTask(RegionCoprocessorEnvironment env, long timeMaxInterval) {
    +            this.env = env;
    +            this.timeMaxInterval = timeMaxInterval;
    +        }
    +  
    +        @Override
    +        public void run() {
    +            PhoenixConnection connForTask = null;
    +            try {
    +                String taskQuery = "SELECT " +
    +                        PhoenixDatabaseMetaData.TASK_TS + ", " +
    +                        PhoenixDatabaseMetaData.TENANT_ID + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_SCHEM + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_NAME +
    +                        " FROM " + PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                        " WHERE "+ PhoenixDatabaseMetaData.TASK_TYPE + " = " + PTable.TaskType.DROP_CHILD_VIEWS.getSerializedValue();
    +
    +                connForTask = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                PreparedStatement taskStatement = connForTask.prepareStatement(taskQuery);
    +                ResultSet rs = taskStatement.executeQuery();
    +                while (rs.next()) {
    +                    try {
    +                        // delete child views only if the parent table is deleted from the system catalog
    +                        Properties tenantProps = new Properties();
    +                        if (rs.getString(3) != null) {
    +                            tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, rs.getString(3));
    +                        }
    +                        PhoenixConnection pconn = QueryUtil.getConnectionOnServer(tenantProps, env.getConfiguration()).unwrap(PhoenixConnection.class);
    +
    +                        MetaDataProtocol.MetaDataMutationResult result = new MetaDataClient(pconn).updateCache(pconn.getTenantId(),
    +                                rs.getString(3), rs.getString(4), true);
    +                        if (result.getMutationCode() != MetaDataProtocol.MutationCode.TABLE_ALREADY_EXISTS) {
    +                            MetaDataEndpointImpl.dropChildViews(env, rs.getBytes(2), rs.getBytes(3), rs.getBytes(4));
    +                        } else if (System.currentTimeMillis() < timeMaxInterval + rs.getTimestamp(1).getTime()) {
    +                            // skip this task as it has not been expired and its parent table has not been dropped yet
    --- End diff --
    
    You can also format the log string with `String.format`.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223883668
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    I don't wanna force here, but I do believe that we should not follow the model that has been established and also to help our future-self :) 


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r228292258
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java ---
    @@ -322,6 +322,12 @@
     
     	public static final String SYSTEM_CATALOG_SPLITTABLE = "phoenix.system.catalog.splittable";
     
    +    // The parameters defined for handling task stored in table SYSTEM.TASK
    +	// The time interval between periodic scans of table SYSTEM.TASK
    +    public static final String TASK_HANDLING_INTERVAL_ATTRIB = "phoenix.task.handling.interval";
    --- End diff --
    
    Will change it to TASK_HANDLING_INTERVAL_MS_ATTRIB


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r224976187
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        Configuration config = env.getConfiguration();
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, String tenantId, String schemaName, String tableName)
    +            throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME + " ( " +
    +                    PhoenixDatabaseMetaData.TASK_TYPE + ", " +
    +                    PhoenixDatabaseMetaData.TENANT_ID + ", " +
    +                    PhoenixDatabaseMetaData.TABLE_SCHEM + ", " +
    +                    PhoenixDatabaseMetaData.TABLE_NAME + " ) VALUES(?,?,?,?)");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            if (tenantId != null) {
    +                stmt.setString(2, tenantId);
    +            }else {
    +                stmt.setNull(2, Types.VARCHAR);
    +            }
    +            stmt.setString(3, schemaName);
    +            stmt.setString(4, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    public static void deleteTask(PhoenixConnection conn, TaskType taskType, Timestamp ts, String tenantId, String schemaName, String tableName) throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("DELETE FROM " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                    " WHERE " + PhoenixDatabaseMetaData.TASK_TYPE + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TASK_TS + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TENANT_ID + (tenantId == null ? " IS NULL " : " = '" + tenantId + "'") + " AND " +
    +                    PhoenixDatabaseMetaData.TABLE_SCHEM + "= ? AND " +
    +                    PhoenixDatabaseMetaData.TABLE_NAME + " = ?");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            stmt.setTimestamp(2, ts);
    +            stmt.setString(3, schemaName);
    +            stmt.setString(4, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    /**
    +     * Task runs periodically to clean up task of child views whose parent is dropped
    +     *
    +     */
    +    public static class DropChildViewsTask extends TimerTask {
    +        private RegionCoprocessorEnvironment env;
    +        private long timeMaxInterval;
    +
    +        public DropChildViewsTask(RegionCoprocessorEnvironment env, long timeMaxInterval) {
    +            this.env = env;
    +            this.timeMaxInterval = timeMaxInterval;
    +        }
    +  
    +        @Override
    +        public void run() {
    +            PhoenixConnection connForTask = null;
    +            Timestamp timestamp = null;
    +            String tenantId = null;
    +            byte[] tenantIdBytes;
    +            String schemaName= null;
    +            byte[] schemaNameBytes;
    +            String tableName = null;
    +            byte[] tableNameBytes;
    +            PhoenixConnection pconn;
    +            try {
    +                String taskQuery = "SELECT " +
    +                        PhoenixDatabaseMetaData.TASK_TS + ", " +
    +                        PhoenixDatabaseMetaData.TENANT_ID + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_SCHEM + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_NAME +
    +                        " FROM " + PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                        " WHERE "+ PhoenixDatabaseMetaData.TASK_TYPE + " = " + PTable.TaskType.DROP_CHILD_VIEWS.getSerializedValue();
    +
    +                connForTask = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                PreparedStatement taskStatement = connForTask.prepareStatement(taskQuery);
    +                ResultSet rs = taskStatement.executeQuery();
    +                while (rs.next()) {
    +                    try {
    +                        // delete child views only if the parent table is deleted from the system catalog
    +                        timestamp = rs.getTimestamp(1);
    +                        tenantId = rs.getString(2);
    +                        tenantIdBytes= rs.getBytes(2);
    +                        schemaName= rs.getString(3);
    +                        schemaNameBytes = rs.getBytes(3);
    +                        tableName= rs.getString(4);
    +                        tableNameBytes = rs.getBytes(4);
    +
    +                        if (tenantId != null) {
    +                            Properties tenantProps = new Properties();
    +                            tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
    +                            pconn = QueryUtil.getConnectionOnServer(tenantProps, env.getConfiguration()).unwrap(PhoenixConnection.class);
    +
    +                        }
    +                        else {
    +                            pconn = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                        }
    +
    +                        MetaDataProtocol.MetaDataMutationResult result = new MetaDataClient(pconn).updateCache(pconn.getTenantId(),
    --- End diff --
    
    @dbwong, I do not think we should be concerned about making a newMetaDataClient per result since the objects of this class have only one instance variable which is a pointer to the PhoenixConnection object that is passed via the constructor. Trying to make one MetaDataClient object per connection is probably as expensive as creating a new object per result. I think it will be over engineering if we do that. 


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223779732
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    --- End diff --
    
    I do not see the need for the suggest name change. I do not see prefix "System" is used commonly. If needed, we may change it to "PhoenixTaskRegionObserver". I have defined a new column called "TaskType" in the same way we defined "LinkType".  I do not see that we need to qualify them with "System".


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by kadirozde <gi...@git.apache.org>.
Github user kadirozde commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223783140
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TaskRegionObserver.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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.phoenix.coprocessor;
    +
    +import java.io.IOException;
    +import java.sql.PreparedStatement;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.TimerTask;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.CoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
    +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
    +import org.apache.hadoop.hbase.coprocessor.RegionObserver;
    +import org.apache.log4j.Level;
    +import org.apache.log4j.Logger;
    +import org.apache.phoenix.cache.GlobalCache;
    +import org.apache.phoenix.jdbc.PhoenixConnection;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +import org.apache.phoenix.jdbc.PhoenixDriver;
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.MetaDataClient;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.PTable.TaskType;
    +
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.QueryUtil;
    +
    +
    +/**
    + * Coprocessor for task related operations. This coprocessor would only be registered
    + * to SYSTEM.TASK table
    + */
    +@SuppressWarnings("deprecation")
    +public class TaskRegionObserver implements RegionObserver, RegionCoprocessor {
    +    public static final Log LOG = LogFactory.getLog(TaskRegionObserver.class);
    +    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(TaskType.values().length);
    +    private long timeInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL;
    +    private long timeMaxInterval = QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL;
    +    @GuardedBy("TaskRegionObserver.class")
    +    // Added for test purposes
    +    private long initialDelay = 0;
    +
    +    @Override
    +    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
    +            boolean abortRequested) {
    +        executor.shutdownNow();
    +        GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().invalidateAll();
    +    }
    +
    +    @Override
    +    public Optional<RegionObserver> getRegionObserver() {
    +        return Optional.of(this);
    +    }
    +
    +    @Override
    +    public void start(CoprocessorEnvironment env) throws IOException {
    +        // sleep a little bit to compensate time clock skew when SYSTEM.CATALOG moves
    +        // among region servers because we relies on server time of RS which is hosting
    +        // SYSTEM.CATALOG
    +        Configuration config = env.getConfiguration();
    +        long sleepTime = config.getLong(QueryServices.CLOCK_SKEW_INTERVAL_ATTRIB,
    +            QueryServicesOptions.DEFAULT_CLOCK_SKEW_INTERVAL);
    +        try {
    +            if(sleepTime > 0) {
    +                Thread.sleep(sleepTime);
    +            }
    +        } catch (InterruptedException ie) {
    +            Thread.currentThread().interrupt();
    +        }
    +        timeInterval =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INTERVAL_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL);
    +        timeMaxInterval =
    +                config.getLong(
    +                        QueryServices.TASK_HANDLING_MAX_INTERVAL_ATTRIB,
    +                        QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL);
    +        initialDelay =
    +                config.getLong(
    +                    QueryServices.TASK_HANDLING_INITIAL_DELAY_ATTRIB,
    +                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INITIAL_DELAY);
    +    }
    +    
    +    @Override
    +    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
    +        final RegionCoprocessorEnvironment env = e.getEnvironment();
    +
    +        // turn off verbose deprecation logging
    +        Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
    +        if (deprecationLogger != null) {
    +            deprecationLogger.setLevel(Level.WARN);
    +        }
    +        try {
    +            Class.forName(PhoenixDriver.class.getName());
    +            DropChildViewsTask task = new DropChildViewsTask(e.getEnvironment(), timeMaxInterval);
    +            executor.scheduleWithFixedDelay(task, initialDelay, timeInterval, TimeUnit.MILLISECONDS);
    +        } catch (ClassNotFoundException ex) {
    +            LOG.error("ScheduleTask cannot start!", ex);
    +        }
    +    }
    +
    +    public static void addTask(PhoenixConnection conn, TaskType taskType, Timestamp ts, String tenantId, String schemaName, String tableName) throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME + " VALUES(?,?,?,?,?)");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            stmt.setTimestamp(2, ts);
    +            if (tenantId != null) {
    +                stmt.setString(3, tenantId);
    +            }else {
    +                stmt.setNull(3, Types.VARCHAR);
    +            }
    +            stmt.setString(4, schemaName);
    +            stmt.setString(5, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    public static void deleteTask(PhoenixConnection conn, TaskType taskType, Timestamp ts, String tenantId, String schemaName, String tableName) throws IOException {
    +        try {
    +            PreparedStatement stmt = conn.prepareStatement("DELETE FROM " +
    +                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                    " WHERE " + PhoenixDatabaseMetaData.TASK_TYPE + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TASK_TS + " = ? AND " +
    +                    PhoenixDatabaseMetaData.TENANT_ID + (tenantId == null ? " IS NULL " : " = '" + tenantId + "'") + " AND " +
    +                    PhoenixDatabaseMetaData.TABLE_SCHEM + "= ? AND " +
    +                    PhoenixDatabaseMetaData.TABLE_NAME + " = ?");
    +            stmt.setByte(1, taskType.getSerializedValue());
    +            stmt.setTimestamp(2, ts);
    +            stmt.setString(3, schemaName);
    +            stmt.setString(4, tableName);
    +            stmt.execute();
    +            conn.commit();
    +        } catch (SQLException e) {
    +            throw new IOException(e);
    +        }
    +    }
    +
    +    /**
    +     * Task runs periodically to clean up task of child views whose parent is dropped
    +     *
    +     */
    +    public static class DropChildViewsTask extends TimerTask {
    +        private RegionCoprocessorEnvironment env;
    +        private long timeMaxInterval;
    +
    +        public DropChildViewsTask(RegionCoprocessorEnvironment env, long timeMaxInterval) {
    +            this.env = env;
    +            this.timeMaxInterval = timeMaxInterval;
    +        }
    +  
    +        @Override
    +        public void run() {
    +            PhoenixConnection connForTask = null;
    +            try {
    +                String taskQuery = "SELECT " +
    +                        PhoenixDatabaseMetaData.TASK_TS + ", " +
    +                        PhoenixDatabaseMetaData.TENANT_ID + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_SCHEM + ", " +
    +                        PhoenixDatabaseMetaData.TABLE_NAME +
    +                        " FROM " + PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
    +                        " WHERE "+ PhoenixDatabaseMetaData.TASK_TYPE + " = " + PTable.TaskType.DROP_CHILD_VIEWS.getSerializedValue();
    +
    +                connForTask = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
    +                PreparedStatement taskStatement = connForTask.prepareStatement(taskQuery);
    +                ResultSet rs = taskStatement.executeQuery();
    +                while (rs.next()) {
    +                    try {
    +                        // delete child views only if the parent table is deleted from the system catalog
    +                        Properties tenantProps = new Properties();
    +                        if (rs.getString(3) != null) {
    +                            tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, rs.getString(3));
    +                        }
    +                        PhoenixConnection pconn = QueryUtil.getConnectionOnServer(tenantProps, env.getConfiguration()).unwrap(PhoenixConnection.class);
    +
    +                        MetaDataProtocol.MetaDataMutationResult result = new MetaDataClient(pconn).updateCache(pconn.getTenantId(),
    +                                rs.getString(3), rs.getString(4), true);
    +                        if (result.getMutationCode() != MetaDataProtocol.MutationCode.TABLE_ALREADY_EXISTS) {
    +                            MetaDataEndpointImpl.dropChildViews(env, rs.getBytes(2), rs.getBytes(3), rs.getBytes(4));
    +                        } else if (System.currentTimeMillis() < timeMaxInterval + rs.getTimestamp(1).getTime()) {
    +                            // skip this task as it has not been expired and its parent table has not been dropped yet
    --- End diff --
    
    I am using mostly strings here, so simple string concatenation works.  I will consider it next time.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r223896074
  
    --- Diff: phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.phoenix.end2end;
    +
    +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.fail;
    +
    +import java.sql.Connection;
    +import java.sql.DriverManager;
    +import java.sql.ResultSet;
    +import java.util.Arrays;
    +import java.util.Collection;
    +
    +import org.apache.hadoop.hbase.HConstants;
    +import org.apache.hadoop.hbase.TableName;
    +import org.apache.phoenix.coprocessor.TableViewFinderResult;
    +import org.apache.phoenix.coprocessor.ViewFinder;
    +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
    +
    +import org.apache.phoenix.query.QueryServices;
    +import org.apache.phoenix.query.QueryServicesOptions;
    +import org.apache.phoenix.schema.PTable;
    +import org.apache.phoenix.schema.TableNotFoundException;
    +import org.apache.phoenix.util.PhoenixRuntime;
    +import org.apache.phoenix.util.SchemaUtil;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.junit.runners.Parameterized.Parameters;
    +
    +@RunWith(Parameterized.class)
    +public class DropTableWithViewsIT extends SplitSystemCatalogIT {
    +
    +    private final boolean isMultiTenant;
    +    private final boolean columnEncoded;
    +    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
    +    
    +    public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
    +        this.isMultiTenant = isMultiTenant;
    +        this.columnEncoded = columnEncoded;
    +    }
    +    
    +    @Parameters(name="DropTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
    +    public static Collection<Boolean[]> data() {
    +        return Arrays.asList(new Boolean[][] { 
    +                { false, false }, { false, true },
    +                { true, false }, { true, true } });
    +    }
    +
    +    private String generateDDL(String format) {
    +        return generateDDL("", format);
    +    }
    +    
    +    private String generateDDL(String options, String format) {
    +        StringBuilder optionsBuilder = new StringBuilder(options);
    +        if (!columnEncoded) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
    +        }
    +        if (isMultiTenant) {
    +            if (optionsBuilder.length()!=0)
    +                optionsBuilder.append(",");
    +            optionsBuilder.append("MULTI_TENANT=true");
    +        }
    +        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
    +            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
    +    }
    +    
    +    @Test
    +    public void testDropTableWithChildViews() throws Exception {
    --- End diff --
    
    Alright, then lets try to refactor it into a couple methods. Will help when reading it.


---

[GitHub] phoenix pull request #359: PHOENIX-4764 - Cleanup metadata of child views fo...

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/phoenix/pull/359#discussion_r222447782
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java ---
    @@ -192,7 +192,39 @@ public static LinkType fromSerializedValue(byte serializedValue) {
                 return LinkType.values()[serializedValue-1];
             }
         }
    -    
    +
    +    public enum TaskType {
    +        DROP_CHILD_VIEWS((byte)1);
    +
    +        private final byte[] byteValue;
    +        private final byte serializedValue;
    +
    +        TaskType(byte serializedValue) {
    +            this.serializedValue = serializedValue;
    +            this.byteValue = Bytes.toBytes(this.name());
    +        }
    +
    +        public byte[] getBytes() {
    +                return byteValue;
    +        }
    +
    +        public byte getSerializedValue() {
    +            return this.serializedValue;
    +        }
    +        public static TaskType getDefault() {
    +            return DROP_CHILD_VIEWS;
    +        }
    +        public static TaskType fromToken(String token) {
    +            return TaskType.valueOf(token.trim().toUpperCase());
    +        }
    +        public static TaskType fromSerializedValue(byte serializedValue) {
    --- End diff --
    
    Why do we need this serialization logic? @twdsilva @kadirozde 
    I see that it has been copied from `LinkType` and other Enum's defined above.


---