You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/07/17 19:23:58 UTC

[GitHub] [kafka] mjsax commented on a change in pull request #8200: KAFKA-5876: IQ should throw different exceptions for different errors(part 1)

mjsax commented on a change in pull request #8200:
URL: https://github.com/apache/kafka/pull/8200#discussion_r456623483



##########
File path: streams/src/main/java/org/apache/kafka/streams/errors/InvalidStateStorePartitionException.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.kafka.streams.errors;
+
+/**
+ * Indicates that the specific state store being queried via
+ * {@link org.apache.kafka.streams.StoreQueryParameters} used an invalid partition.

Review comment:
       It's a little bit unclear what "invalid" means. Maybe better:
   ```
   ...used a partitioning that is not assigned to this instance. You can use {@link KafkaStreams#allMetadata()} to discover the correct instance that hosts the requested partition.
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/errors/StreamsNotStartedException.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.kafka.streams.errors;
+
+/**
+ * Indicate query a state store when Kafka Streams state is {@link org.apache.kafka.streams.KafkaStreams.State#CREATED CREATED}.
+ * User can just retry and wait until to {@link org.apache.kafka.streams.KafkaStreams.State#RUNNING RUNNING}

Review comment:
       `To query state stores, it's required to first start Kafka Streams via {@link KafkaStreams#start()}. You can retry to query the state after the state transitioned to ...`
   

##########
File path: streams/src/main/java/org/apache/kafka/streams/errors/InvalidStateStoreException.java
##########
@@ -18,13 +18,11 @@
 
 
 /**
- * Indicates that there was a problem when trying to access a
- * {@link org.apache.kafka.streams.processor.StateStore StateStore}, i.e, the Store is no longer valid because it is
- * closed or doesn't exist any more due to a rebalance.
- * <p>
- * These exceptions may be transient, i.e., during a rebalance it won't be possible to query the stores as they are
- * being (re)-initialized. Once the rebalance has completed the stores will be available again. Hence, it is valid
- * to backoff and retry when handling this exception.
+ * <p>Indicates that there was a problem when trying to access a {@link org.apache.kafka.streams.processor.StateStore StateStore}.
+ * InvalidStateStoreException not thrown directly but only following sub-classes:</p>

Review comment:
       nit: JavaDocs does support HTML, but `<p>` is actually not HTML but a JavaDoc annotation and the closing tag `</p>` is not required. In fact, we only have a single paragraph and thus don't need any tags at all.
   
   `InvalidStateStoreException` -> `{@code InvalidStateStoreException}`
   
   `...but only it subclasses.`
   
   I would not list the sub-classes because the JavaDocs will link to them automatically. Also, if we add new sub-classes in the future, I am sure we would forget to update the JavaDocs here. If we don' list them, we can't forget to update as there is nothing to be updated. :)

##########
File path: streams/src/main/java/org/apache/kafka/streams/errors/StateStoreMigratedException.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.kafka.streams.errors;
+
+/**
+ * Indicates that the state store being queried is closed although the Kafka Streams state is
+ * {@link org.apache.kafka.streams.KafkaStreams.State#RUNNING RUNNING} or
+ * {@link org.apache.kafka.streams.KafkaStreams.State#REBALANCING REBALANCING}.
+ * It could happen because the partition moved to some other instance during a rebalance so

Review comment:
       `It` -> `This`
   
   `partition` -> `store` ?

##########
File path: streams/src/main/java/org/apache/kafka/streams/errors/StateStoreNotAvailableException.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.kafka.streams.errors;
+
+/**
+ * Indicates that the state store being queried is already closed. It could happen when Kafka Streams is in

Review comment:
       `It` -> `This`

##########
File path: streams/src/main/java/org/apache/kafka/streams/errors/StreamsRebalancingException.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.kafka.streams.errors;
+
+/**
+ * Indicate query a state store and stream thread state is not running when Kafka Streams state is

Review comment:
       `Indicate[s] that Kafka Streams is in state {@link org.apache.kafka.streams.KafkaStreams.State#REBALANCING REBALANCING} and thus cannot be queried by default.
   You can retry to query after the rebalance finished. As an alternative, you can also query (potentially stale) state stores during a rebalance via {@link StoreQueryParameters#enableStaleStores()}.`

##########
File path: streams/src/main/java/org/apache/kafka/streams/errors/StreamsNotStartedException.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.kafka.streams.errors;
+
+/**
+ * Indicate query a state store when Kafka Streams state is {@link org.apache.kafka.streams.KafkaStreams.State#CREATED CREATED}.

Review comment:
       `Indicate[s] that Kafka Streams is in state {@link org.apache.kafka.streams.KafkaStreams.State#CREATED CREATED} and thus state stores cannot be queries yet.`

##########
File path: streams/src/main/java/org/apache/kafka/streams/errors/UnknownStateStoreException.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.kafka.streams.errors;
+
+/**
+ * Indicates that the state store being queried is invalid. Hence, it will be futile to retry again.

Review comment:
       `invalid` -> `unknown, i.e., the state store does either not exist in your topology or it is not queryable.`
   
   I would remove `Hence, it will be futile to retry again.`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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