You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@phoenix.apache.org by Jacobo Coll <ja...@gmail.com> on 2018/02/13 16:09:04 UTC

Drop column timeout

Hi all,

I have a table in phoenix with 100M rows and ~3000 columns. I am trying to
remove some columns, but after some seconds, it fails with a timeout
exception:


0: jdbc:phoenix:> ALTER TABLE "ns"."table" DROP COLUMN IF EXISTS "myColumn";
Error: org.apache.phoenix.exception.PhoenixIOException: Failed to get
result within timeout, timeout=60000ms (state=08000,code=101)
org.apache.phoenix.exception.PhoenixIOException:
org.apache.phoenix.exception.PhoenixIOException: Failed to get result
within timeout, timeout=60000ms
        at
org.apache.phoenix.util.ServerUtil.parseServerException(ServerUtil.java:111)
        at
org.apache.phoenix.iterate.BaseResultIterators.getIterators(BaseResultIterators.java:771)
        at
org.apache.phoenix.iterate.BaseResultIterators.getIterators(BaseResultIterators.java:714)
        at
org.apache.phoenix.iterate.ConcatResultIterator.getIterators(ConcatResultIterator.java:50)
        at
org.apache.phoenix.iterate.ConcatResultIterator.currentIterator(ConcatResultIterator.java:97)
        at
org.apache.phoenix.iterate.ConcatResultIterator.next(ConcatResultIterator.java:117)
        at
org.apache.phoenix.iterate.BaseGroupedAggregatingResultIterator.next(BaseGroupedAggregatingResultIterator.java:64)
        at
org.apache.phoenix.iterate.UngroupedAggregatingResultIterator.next(UngroupedAggregatingResultIterator.java:39)
        at
org.apache.phoenix.compile.PostDDLCompiler$2.execute(PostDDLCompiler.java:285)
        at
org.apache.phoenix.query.ConnectionQueryServicesImpl.updateData(ConnectionQueryServicesImpl.java:2823)
        at
org.apache.phoenix.schema.MetaDataClient.dropColumn(MetaDataClient.java:3209)
        at
org.apache.phoenix.jdbc.PhoenixStatement$ExecutableDropColumnStatement$1.execute(PhoenixStatement.java:1127)
        at
org.apache.phoenix.jdbc.PhoenixStatement$2.call(PhoenixStatement.java:343)
        at
org.apache.phoenix.jdbc.PhoenixStatement$2.call(PhoenixStatement.java:331)
        at org.apache.phoenix.call.CallRunner.run(CallRunner.java:53)
        at
org.apache.phoenix.jdbc.PhoenixStatement.executeMutation(PhoenixStatement.java:329)
        at
org.apache.phoenix.jdbc.PhoenixStatement.execute(PhoenixStatement.java:1440)
        at sqlline.Commands.execute(Commands.java:822)
        at sqlline.Commands.sql(Commands.java:732)
        at sqlline.SqlLine.dispatch(SqlLine.java:808)
        at sqlline.SqlLine.begin(SqlLine.java:681)
        at sqlline.SqlLine.start(SqlLine.java:398)
        at sqlline.SqlLine.main(SqlLine.java:292)
Caused by: java.util.concurrent.ExecutionException:
org.apache.phoenix.exception.PhoenixIOException: Failed to get result
within timeout, timeout=60000ms
        at java.util.concurrent.FutureTask.report(FutureTask.java:122)
        at java.util.concurrent.FutureTask.get(FutureTask.java:206)
        at
org.apache.phoenix.iterate.BaseResultIterators.getIterators(BaseResultIterators.java:766)
        ... 21 more
Caused by: org.apache.phoenix.exception.PhoenixIOException: Failed to get
result within timeout, timeout=60000ms
        at
org.apache.phoenix.util.ServerUtil.parseServerException(ServerUtil.java:111)
        at
org.apache.phoenix.iterate.TableResultIterator.initScanner(TableResultIterator.java:203)
        at
org.apache.phoenix.iterate.ParallelIterators$1.call(ParallelIterators.java:108)
        at
org.apache.phoenix.iterate.ParallelIterators$1.call(ParallelIterators.java:103)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at
org.apache.phoenix.job.JobManager$InstrumentedJobFutureTask.run(JobManager.java:183)
        at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.io.IOException: Failed to get result within timeout,
timeout=60000ms
        at
org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:206)
        at
org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:60)
        at
org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:200)
        at
org.apache.hadoop.hbase.client.ClientScanner.call(ClientScanner.java:326)
        at
org.apache.hadoop.hbase.client.ClientScanner.nextScanner(ClientScanner.java:301)
        at
org.apache.hadoop.hbase.client.ClientScanner.initializeScannerInConstruction(ClientScanner.java:166)
        at
org.apache.hadoop.hbase.client.ClientScanner.<init>(ClientScanner.java:161)
        at org.apache.hadoop.hbase.client.HTable.getScanner(HTable.java:794)
        at
org.apache.phoenix.iterate.TableResultIterator.initScanner(TableResultIterator.java:199)
        ... 7 more



After this, the column is partially removed. Throws a "Error: ERROR 504
(42703): Undefined column. columnName=myColumn (state=42703,code=504)", but
I can see values when adding this column as *Dynamic Column*.

More weird is the fact that the column is still registered in the
SYSTEM.CATALOG table.

I suppose that this could be solved increasing the timeouts, but that will
be just delay the issue.

- Is there any way of removing a column asynchronously, like when creating
indexes?
- If not, can I just remove that column from the metadata (as if it was a
VIEW instead of a TABLE), and then remove the data using the HBase API?


I am using Hortonworks HDP 2.5.0:

   - Phoenix 4.7
   - HBase 1.1.2



Regards,
Jacobo Coll

Re: Drop column timeout

Posted by James Taylor <ja...@apache.org>.
Hi Jacobo,
Please file a JIRA for asynchronous drop column functionality. There's a
few ways that could be implemented. We could execute the call that issues
the delete markers on the server-side in a separate thread (similar to what
we do with UPDATE STATISTICS), or we could support a map-reduce job that
executes the drop column. The former is easier, but wouldn't be resilient
against the cluster being bounced. It'd also put more load on the cluster
(but would execute faster). The latter would be a bit more work and with
the current framework, the MR job would need to be started manually.

You could remove the column manually from the SYSTEM.CATALOG, but you'd
want to bounce the region server hosting this table so that the cache gets
cleared (or call MetaDataService.clearTableFromCache() for the table).

Thanks,
James

On Tue, Feb 13, 2018 at 8:59 AM, Jacobo Coll <ja...@gmail.com> wrote:

> Hi Flavio,
>
> I was trying to find a different solution here. This doesn't seem like a
> long term solution, as I expect the table to increase, and these new
> timeouts may not be enough in the future. Also, I don't feel comfortable
> increasing the timeouts that much.
>
>
> - Is there any way of removing a column asynchronously, like when creating
>> indexes?
>> - If not, can I just remove that column from the metadata (as if it was a
>> VIEW instead of a TABLE), and then remove the data using the HBase API?
>>
>
>
> Thanks
>
> 2018-02-13 16:18 GMT+00:00 Flavio Pompermaier <po...@okkam.it>:
>
>> I also had similar troubles and I fixed them changing the following
>> params (both on server and client side and restarting hbase):
>>
>> hbase.rpc.timeout (to 600000)
>> phoenix.query.timeoutMs (to 600000)
>> hbase.client.scanner.timeout.period (from 1 m to 10m)
>> hbase.regionserver.lease.period (from 1 m to 10m)
>>
>> I hope this could help you!
>>
>>
>> On Tue, Feb 13, 2018 at 5:09 PM, Jacobo Coll <ja...@gmail.com> wrote:
>>
>>> Hi all,
>>>
>>> I have a table in phoenix with 100M rows and ~3000 columns. I am trying
>>> to remove some columns, but after some seconds, it fails with a timeout
>>> exception:
>>>
>>>
>>> 0: jdbc:phoenix:> ALTER TABLE "ns"."table" DROP COLUMN IF EXISTS
>>> "myColumn";
>>> Error: org.apache.phoenix.exception.PhoenixIOException: Failed to get
>>> result within timeout, timeout=60000ms (state=08000,code=101)
>>> org.apache.phoenix.exception.PhoenixIOException:
>>> org.apache.phoenix.exception.PhoenixIOException: Failed to get result
>>> within timeout, timeout=60000ms
>>>         at org.apache.phoenix.util.ServerUtil.parseServerException(Serv
>>> erUtil.java:111)
>>>         at org.apache.phoenix.iterate.BaseResultIterators.getIterators(
>>> BaseResultIterators.java:771)
>>>         at org.apache.phoenix.iterate.BaseResultIterators.getIterators(
>>> BaseResultIterators.java:714)
>>>         at org.apache.phoenix.iterate.ConcatResultIterator.getIterators
>>> (ConcatResultIterator.java:50)
>>>         at org.apache.phoenix.iterate.ConcatResultIterator.currentItera
>>> tor(ConcatResultIterator.java:97)
>>>         at org.apache.phoenix.iterate.ConcatResultIterator.next(ConcatR
>>> esultIterator.java:117)
>>>         at org.apache.phoenix.iterate.BaseGroupedAggregatingResultItera
>>> tor.next(BaseGroupedAggregatingResultIterator.java:64)
>>>         at org.apache.phoenix.iterate.UngroupedAggregatingResultIterato
>>> r.next(UngroupedAggregatingResultIterator.java:39)
>>>         at org.apache.phoenix.compile.PostDDLCompiler$2.execute(PostDDL
>>> Compiler.java:285)
>>>         at org.apache.phoenix.query.ConnectionQueryServicesImpl.updateD
>>> ata(ConnectionQueryServicesImpl.java:2823)
>>>         at org.apache.phoenix.schema.MetaDataClient.dropColumn(MetaData
>>> Client.java:3209)
>>>         at org.apache.phoenix.jdbc.PhoenixStatement$ExecutableDropColum
>>> nStatement$1.execute(PhoenixStatement.java:1127)
>>>         at org.apache.phoenix.jdbc.PhoenixStatement$2.call(PhoenixState
>>> ment.java:343)
>>>         at org.apache.phoenix.jdbc.PhoenixStatement$2.call(PhoenixState
>>> ment.java:331)
>>>         at org.apache.phoenix.call.CallRunner.run(CallRunner.java:53)
>>>         at org.apache.phoenix.jdbc.PhoenixStatement.executeMutation(Pho
>>> enixStatement.java:329)
>>>         at org.apache.phoenix.jdbc.PhoenixStatement.execute(PhoenixStat
>>> ement.java:1440)
>>>         at sqlline.Commands.execute(Commands.java:822)
>>>         at sqlline.Commands.sql(Commands.java:732)
>>>         at sqlline.SqlLine.dispatch(SqlLine.java:808)
>>>         at sqlline.SqlLine.begin(SqlLine.java:681)
>>>         at sqlline.SqlLine.start(SqlLine.java:398)
>>>         at sqlline.SqlLine.main(SqlLine.java:292)
>>> Caused by: java.util.concurrent.ExecutionException:
>>> org.apache.phoenix.exception.PhoenixIOException: Failed to get result
>>> within timeout, timeout=60000ms
>>>         at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>>>         at java.util.concurrent.FutureTask.get(FutureTask.java:206)
>>>         at org.apache.phoenix.iterate.BaseResultIterators.getIterators(
>>> BaseResultIterators.java:766)
>>>         ... 21 more
>>> Caused by: org.apache.phoenix.exception.PhoenixIOException: Failed to
>>> get result within timeout, timeout=60000ms
>>>         at org.apache.phoenix.util.ServerUtil.parseServerException(Serv
>>> erUtil.java:111)
>>>         at org.apache.phoenix.iterate.TableResultIterator.initScanner(T
>>> ableResultIterator.java:203)
>>>         at org.apache.phoenix.iterate.ParallelIterators$1.call(Parallel
>>> Iterators.java:108)
>>>         at org.apache.phoenix.iterate.ParallelIterators$1.call(Parallel
>>> Iterators.java:103)
>>>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>         at org.apache.phoenix.job.JobManager$InstrumentedJobFutureTask.
>>> run(JobManager.java:183)
>>>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPool
>>> Executor.java:1142)
>>>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoo
>>> lExecutor.java:617)
>>>         at java.lang.Thread.run(Thread.java:745)
>>> Caused by: java.io.IOException: Failed to get result within timeout,
>>> timeout=60000ms
>>>         at org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.c
>>> all(ScannerCallableWithReplicas.java:206)
>>>         at org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.c
>>> all(ScannerCallableWithReplicas.java:60)
>>>         at org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithout
>>> Retries(RpcRetryingCaller.java:200)
>>>         at org.apache.hadoop.hbase.client.ClientScanner.call(ClientScan
>>> ner.java:326)
>>>         at org.apache.hadoop.hbase.client.ClientScanner.nextScanner(Cli
>>> entScanner.java:301)
>>>         at org.apache.hadoop.hbase.client.ClientScanner.initializeScann
>>> erInConstruction(ClientScanner.java:166)
>>>         at org.apache.hadoop.hbase.client.ClientScanner.<init>(ClientSc
>>> anner.java:161)
>>>         at org.apache.hadoop.hbase.client.HTable.getScanner(HTable.java
>>> :794)
>>>         at org.apache.phoenix.iterate.TableResultIterator.initScanner(T
>>> ableResultIterator.java:199)
>>>         ... 7 more
>>>
>>>
>>>
>>> After this, the column is partially removed. Throws a "Error: ERROR 504
>>> (42703): Undefined column. columnName=myColumn (state=42703,code=504)",
>>> but I can see values when adding this column as *Dynamic Column*.
>>>
>>> More weird is the fact that the column is still registered in the
>>> SYSTEM.CATALOG table.
>>>
>>> I suppose that this could be solved increasing the timeouts, but that
>>> will be just delay the issue.
>>>
>>> - Is there any way of removing a column asynchronously, like when
>>> creating indexes?
>>> - If not, can I just remove that column from the metadata (as if it was
>>> a VIEW instead of a TABLE), and then remove the data using the HBase API?
>>>
>>>
>>> I am using Hortonworks HDP 2.5.0:
>>>
>>>    - Phoenix 4.7
>>>    - HBase 1.1.2
>>>
>>>
>>>
>>> Regards,
>>> Jacobo Coll
>>>
>>
>>
>

Re: Drop column timeout

Posted by Jacobo Coll <ja...@gmail.com>.
Hi Flavio,

I was trying to find a different solution here. This doesn't seem like a
long term solution, as I expect the table to increase, and these new
timeouts may not be enough in the future. Also, I don't feel comfortable
increasing the timeouts that much.


- Is there any way of removing a column asynchronously, like when creating
> indexes?
> - If not, can I just remove that column from the metadata (as if it was a
> VIEW instead of a TABLE), and then remove the data using the HBase API?
>


Thanks

2018-02-13 16:18 GMT+00:00 Flavio Pompermaier <po...@okkam.it>:

> I also had similar troubles and I fixed them changing the following
> params (both on server and client side and restarting hbase):
>
> hbase.rpc.timeout (to 600000)
> phoenix.query.timeoutMs (to 600000)
> hbase.client.scanner.timeout.period (from 1 m to 10m)
> hbase.regionserver.lease.period (from 1 m to 10m)
>
> I hope this could help you!
>
>
> On Tue, Feb 13, 2018 at 5:09 PM, Jacobo Coll <ja...@gmail.com> wrote:
>
>> Hi all,
>>
>> I have a table in phoenix with 100M rows and ~3000 columns. I am trying
>> to remove some columns, but after some seconds, it fails with a timeout
>> exception:
>>
>>
>> 0: jdbc:phoenix:> ALTER TABLE "ns"."table" DROP COLUMN IF EXISTS
>> "myColumn";
>> Error: org.apache.phoenix.exception.PhoenixIOException: Failed to get
>> result within timeout, timeout=60000ms (state=08000,code=101)
>> org.apache.phoenix.exception.PhoenixIOException:
>> org.apache.phoenix.exception.PhoenixIOException: Failed to get result
>> within timeout, timeout=60000ms
>>         at org.apache.phoenix.util.ServerUtil.parseServerException(Serv
>> erUtil.java:111)
>>         at org.apache.phoenix.iterate.BaseResultIterators.getIterators(
>> BaseResultIterators.java:771)
>>         at org.apache.phoenix.iterate.BaseResultIterators.getIterators(
>> BaseResultIterators.java:714)
>>         at org.apache.phoenix.iterate.ConcatResultIterator.getIterators
>> (ConcatResultIterator.java:50)
>>         at org.apache.phoenix.iterate.ConcatResultIterator.currentItera
>> tor(ConcatResultIterator.java:97)
>>         at org.apache.phoenix.iterate.ConcatResultIterator.next(ConcatR
>> esultIterator.java:117)
>>         at org.apache.phoenix.iterate.BaseGroupedAggregatingResultItera
>> tor.next(BaseGroupedAggregatingResultIterator.java:64)
>>         at org.apache.phoenix.iterate.UngroupedAggregatingResultIterato
>> r.next(UngroupedAggregatingResultIterator.java:39)
>>         at org.apache.phoenix.compile.PostDDLCompiler$2.execute(PostDDL
>> Compiler.java:285)
>>         at org.apache.phoenix.query.ConnectionQueryServicesImpl.updateD
>> ata(ConnectionQueryServicesImpl.java:2823)
>>         at org.apache.phoenix.schema.MetaDataClient.dropColumn(MetaData
>> Client.java:3209)
>>         at org.apache.phoenix.jdbc.PhoenixStatement$ExecutableDropColum
>> nStatement$1.execute(PhoenixStatement.java:1127)
>>         at org.apache.phoenix.jdbc.PhoenixStatement$2.call(PhoenixState
>> ment.java:343)
>>         at org.apache.phoenix.jdbc.PhoenixStatement$2.call(PhoenixState
>> ment.java:331)
>>         at org.apache.phoenix.call.CallRunner.run(CallRunner.java:53)
>>         at org.apache.phoenix.jdbc.PhoenixStatement.executeMutation(Pho
>> enixStatement.java:329)
>>         at org.apache.phoenix.jdbc.PhoenixStatement.execute(PhoenixStat
>> ement.java:1440)
>>         at sqlline.Commands.execute(Commands.java:822)
>>         at sqlline.Commands.sql(Commands.java:732)
>>         at sqlline.SqlLine.dispatch(SqlLine.java:808)
>>         at sqlline.SqlLine.begin(SqlLine.java:681)
>>         at sqlline.SqlLine.start(SqlLine.java:398)
>>         at sqlline.SqlLine.main(SqlLine.java:292)
>> Caused by: java.util.concurrent.ExecutionException:
>> org.apache.phoenix.exception.PhoenixIOException: Failed to get result
>> within timeout, timeout=60000ms
>>         at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>>         at java.util.concurrent.FutureTask.get(FutureTask.java:206)
>>         at org.apache.phoenix.iterate.BaseResultIterators.getIterators(
>> BaseResultIterators.java:766)
>>         ... 21 more
>> Caused by: org.apache.phoenix.exception.PhoenixIOException: Failed to
>> get result within timeout, timeout=60000ms
>>         at org.apache.phoenix.util.ServerUtil.parseServerException(Serv
>> erUtil.java:111)
>>         at org.apache.phoenix.iterate.TableResultIterator.initScanner(T
>> ableResultIterator.java:203)
>>         at org.apache.phoenix.iterate.ParallelIterators$1.call(Parallel
>> Iterators.java:108)
>>         at org.apache.phoenix.iterate.ParallelIterators$1.call(Parallel
>> Iterators.java:103)
>>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>         at org.apache.phoenix.job.JobManager$InstrumentedJobFutureTask.
>> run(JobManager.java:183)
>>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPool
>> Executor.java:1142)
>>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoo
>> lExecutor.java:617)
>>         at java.lang.Thread.run(Thread.java:745)
>> Caused by: java.io.IOException: Failed to get result within timeout,
>> timeout=60000ms
>>         at org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.c
>> all(ScannerCallableWithReplicas.java:206)
>>         at org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.c
>> all(ScannerCallableWithReplicas.java:60)
>>         at org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithout
>> Retries(RpcRetryingCaller.java:200)
>>         at org.apache.hadoop.hbase.client.ClientScanner.call(ClientScan
>> ner.java:326)
>>         at org.apache.hadoop.hbase.client.ClientScanner.nextScanner(
>> ClientScanner.java:301)
>>         at org.apache.hadoop.hbase.client.ClientScanner.initializeScann
>> erInConstruction(ClientScanner.java:166)
>>         at org.apache.hadoop.hbase.client.ClientScanner.<init>(ClientSc
>> anner.java:161)
>>         at org.apache.hadoop.hbase.client.HTable.getScanner(HTable.
>> java:794)
>>         at org.apache.phoenix.iterate.TableResultIterator.initScanner(T
>> ableResultIterator.java:199)
>>         ... 7 more
>>
>>
>>
>> After this, the column is partially removed. Throws a "Error: ERROR 504
>> (42703): Undefined column. columnName=myColumn (state=42703,code=504)",
>> but I can see values when adding this column as *Dynamic Column*.
>>
>> More weird is the fact that the column is still registered in the
>> SYSTEM.CATALOG table.
>>
>> I suppose that this could be solved increasing the timeouts, but that
>> will be just delay the issue.
>>
>> - Is there any way of removing a column asynchronously, like when
>> creating indexes?
>> - If not, can I just remove that column from the metadata (as if it was a
>> VIEW instead of a TABLE), and then remove the data using the HBase API?
>>
>>
>> I am using Hortonworks HDP 2.5.0:
>>
>>    - Phoenix 4.7
>>    - HBase 1.1.2
>>
>>
>>
>> Regards,
>> Jacobo Coll
>>
>
>

Re: Drop column timeout

Posted by Flavio Pompermaier <po...@okkam.it>.
I also had similar troubles and I fixed them changing the following params
(both on server and client side and restarting hbase):

hbase.rpc.timeout (to 600000)
phoenix.query.timeoutMs (to 600000)
hbase.client.scanner.timeout.period (from 1 m to 10m)
hbase.regionserver.lease.period (from 1 m to 10m)

I hope this could help you!


On Tue, Feb 13, 2018 at 5:09 PM, Jacobo Coll <ja...@gmail.com> wrote:

> Hi all,
>
> I have a table in phoenix with 100M rows and ~3000 columns. I am trying to
> remove some columns, but after some seconds, it fails with a timeout
> exception:
>
>
> 0: jdbc:phoenix:> ALTER TABLE "ns"."table" DROP COLUMN IF EXISTS
> "myColumn";
> Error: org.apache.phoenix.exception.PhoenixIOException: Failed to get
> result within timeout, timeout=60000ms (state=08000,code=101)
> org.apache.phoenix.exception.PhoenixIOException:
> org.apache.phoenix.exception.PhoenixIOException: Failed to get result
> within timeout, timeout=60000ms
>         at org.apache.phoenix.util.ServerUtil.parseServerException(
> ServerUtil.java:111)
>         at org.apache.phoenix.iterate.BaseResultIterators.getIterators(
> BaseResultIterators.java:771)
>         at org.apache.phoenix.iterate.BaseResultIterators.getIterators(
> BaseResultIterators.java:714)
>         at org.apache.phoenix.iterate.ConcatResultIterator.getIterators(
> ConcatResultIterator.java:50)
>         at org.apache.phoenix.iterate.ConcatResultIterator.
> currentIterator(ConcatResultIterator.java:97)
>         at org.apache.phoenix.iterate.ConcatResultIterator.next(
> ConcatResultIterator.java:117)
>         at org.apache.phoenix.iterate.BaseGroupedAggregatingResultIt
> erator.next(BaseGroupedAggregatingResultIterator.java:64)
>         at org.apache.phoenix.iterate.UngroupedAggregatingResultIter
> ator.next(UngroupedAggregatingResultIterator.java:39)
>         at org.apache.phoenix.compile.PostDDLCompiler$2.execute(
> PostDDLCompiler.java:285)
>         at org.apache.phoenix.query.ConnectionQueryServicesImpl.
> updateData(ConnectionQueryServicesImpl.java:2823)
>         at org.apache.phoenix.schema.MetaDataClient.dropColumn(
> MetaDataClient.java:3209)
>         at org.apache.phoenix.jdbc.PhoenixStatement$
> ExecutableDropColumnStatement$1.execute(PhoenixStatement.java:1127)
>         at org.apache.phoenix.jdbc.PhoenixStatement$2.call(
> PhoenixStatement.java:343)
>         at org.apache.phoenix.jdbc.PhoenixStatement$2.call(
> PhoenixStatement.java:331)
>         at org.apache.phoenix.call.CallRunner.run(CallRunner.java:53)
>         at org.apache.phoenix.jdbc.PhoenixStatement.executeMutation(
> PhoenixStatement.java:329)
>         at org.apache.phoenix.jdbc.PhoenixStatement.execute(
> PhoenixStatement.java:1440)
>         at sqlline.Commands.execute(Commands.java:822)
>         at sqlline.Commands.sql(Commands.java:732)
>         at sqlline.SqlLine.dispatch(SqlLine.java:808)
>         at sqlline.SqlLine.begin(SqlLine.java:681)
>         at sqlline.SqlLine.start(SqlLine.java:398)
>         at sqlline.SqlLine.main(SqlLine.java:292)
> Caused by: java.util.concurrent.ExecutionException:
> org.apache.phoenix.exception.PhoenixIOException: Failed to get result
> within timeout, timeout=60000ms
>         at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>         at java.util.concurrent.FutureTask.get(FutureTask.java:206)
>         at org.apache.phoenix.iterate.BaseResultIterators.getIterators(
> BaseResultIterators.java:766)
>         ... 21 more
> Caused by: org.apache.phoenix.exception.PhoenixIOException: Failed to get
> result within timeout, timeout=60000ms
>         at org.apache.phoenix.util.ServerUtil.parseServerException(
> ServerUtil.java:111)
>         at org.apache.phoenix.iterate.TableResultIterator.initScanner(
> TableResultIterator.java:203)
>         at org.apache.phoenix.iterate.ParallelIterators$1.call(
> ParallelIterators.java:108)
>         at org.apache.phoenix.iterate.ParallelIterators$1.call(
> ParallelIterators.java:103)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>         at org.apache.phoenix.job.JobManager$
> InstrumentedJobFutureTask.run(JobManager.java:183)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(
> ThreadPoolExecutor.java:1142)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(
> ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.IOException: Failed to get result within timeout,
> timeout=60000ms
>         at org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.
> call(ScannerCallableWithReplicas.java:206)
>         at org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.
> call(ScannerCallableWithReplicas.java:60)
>         at org.apache.hadoop.hbase.client.RpcRetryingCaller.
> callWithoutRetries(RpcRetryingCaller.java:200)
>         at org.apache.hadoop.hbase.client.ClientScanner.call(
> ClientScanner.java:326)
>         at org.apache.hadoop.hbase.client.ClientScanner.
> nextScanner(ClientScanner.java:301)
>         at org.apache.hadoop.hbase.client.ClientScanner.
> initializeScannerInConstruction(ClientScanner.java:166)
>         at org.apache.hadoop.hbase.client.ClientScanner.<init>(
> ClientScanner.java:161)
>         at org.apache.hadoop.hbase.client.HTable.getScanner(
> HTable.java:794)
>         at org.apache.phoenix.iterate.TableResultIterator.initScanner(
> TableResultIterator.java:199)
>         ... 7 more
>
>
>
> After this, the column is partially removed. Throws a "Error: ERROR 504
> (42703): Undefined column. columnName=myColumn (state=42703,code=504)",
> but I can see values when adding this column as *Dynamic Column*.
>
> More weird is the fact that the column is still registered in the
> SYSTEM.CATALOG table.
>
> I suppose that this could be solved increasing the timeouts, but that will
> be just delay the issue.
>
> - Is there any way of removing a column asynchronously, like when creating
> indexes?
> - If not, can I just remove that column from the metadata (as if it was a
> VIEW instead of a TABLE), and then remove the data using the HBase API?
>
>
> I am using Hortonworks HDP 2.5.0:
>
>    - Phoenix 4.7
>    - HBase 1.1.2
>
>
>
> Regards,
> Jacobo Coll
>