You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by StephanEwen <gi...@git.apache.org> on 2017/02/20 17:32:43 UTC

[GitHub] flink pull request #3368: [FLINK-5854] [core] Add base Flink Exception class...

GitHub user StephanEwen opened a pull request:

    https://github.com/apache/flink/pull/3368

    [FLINK-5854] [core] Add base Flink Exception classes

    This pull request adds two exception base classes: `FlinkException` and `FlinkRuntimeException`.
    They are useful in improving the way certain parts of the code handle exceptions.
    
      - `FlinkException` is a base class for checked exceptions that indicate that something related to using Flink went wrong. It is helpful, because letting a method throw `FlinkException` rather than `Exception` already helps to not include all of Java's runtime exceptions, which indicate programming errors, rather than situations that should be recovered.
      - `FlinkRuntimeException` as a Flink-specific subclass of `RuntimeException` comes in handy in places where no exceptions were declared, for example when reusing an interface that does not declare exceptions.
    
    **Important: This does not mean we should just declare `FlinkException` everywhere and throw and catch `FlinkException` and `FlinkRuntimeException` arbitrarily. Exception handling remains a careful and conscious task.**
    
    This also adds the `DynamicCodeLoadingException` subclass of `FlinkException` as an example.

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

    $ git pull https://github.com/StephanEwen/incubator-flink exceptions

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

    https://github.com/apache/flink/pull/3368.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 #3368
    
----
commit 1bed2d20a5ccfae4ae7bdfadaaf03fcbe1dba449
Author: Stephan Ewen <se...@apache.org>
Date:   2017-02-17T15:24:35Z

    [FLINK-XXXX] [core] Add base Flink Exception classes

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3368: [FLINK-5854] [core] Add base Flink Exception class...

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

    https://github.com/apache/flink/pull/3368#discussion_r102431100
  
    --- Diff: flink-core/src/main/java/org/apache/flink/util/DynamicCodeLoadingException.java ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.flink.util;
    +
    +import org.apache.flink.annotation.Public;
    +
    +/**
    + * An exception that is thrown if the dynamic instantiation of code fails.
    + * 
    + * <p>This exception is supposed to "sum up" the zoo of exceptions typically thrown around
    + * dynamic code loading and instantiations:
    + * 
    + * <pre>{@code
    + * try {
    + *     Class.forName(classname).asSubclass(TheType.class).newInstance();
    + * }
    + * catch (ClassNotFoundException | ClassCastException | InstantiationException | IllegalAccessException e) {
    + *     throw new DynamicCodeLoadingException(e);
    + * }
    + * }</pre>
    + */
    +@Public
    +public class DynamicCodeLoadingException extends FlinkException {
    +
    +	private static final long serialVersionUID = -25138443817255490L;
    +
    +	/**
    +	 * Creates a new exception with the given message and cause
    +	 *
    +	 * @param message The exception message
    +	 * @param cause The exception that caused this exception
    +	 */
    +	public DynamicCodeLoadingException(String message, Throwable cause) {
    --- End diff --
    
    There may be other cases for this exception that we do not anticipate, like for shipping of generated code in bytes that is dynamically converted to classes (not sure what exceptions that will entail, but probably `ClassFormatError` and so on).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3368: [FLINK-5854] [core] Add base Flink Exception class...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/3368


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3368: [FLINK-5854] [core] Add base Flink Exception classes

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/3368
  
    Could you name 1 or 2 examples for situations where you think it is appropriate to throw a ```FlinkException```? Would invalid arguments (like a String being null) be a reason to do so?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3368: [FLINK-5854] [core] Add base Flink Exception classes

Posted by uce <gi...@git.apache.org>.
Github user uce commented on the issue:

    https://github.com/apache/flink/pull/3368
  
    Hey Stephan! The changes look very good. Thanks also for your explanations, I think this is something that we should focus on more during code reviews. Actually, a section in the Internals or Contribution docs or Wiki would be helpful.
    
    A question by example:
    Curator `throws Exception` on most operations. Previously, this was simply forwarded. Should this now be wrapped in a `FlinkRuntimeException`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3368: [FLINK-5854] [core] Add base Flink Exception classes

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/3368
  
    @zentol There are many places in the runtime that declare `throws Exception`, for example virtually all of the state handling code. This always came from the desire to throw `IOException` plus something that expresses that non-I/O stuff related to Flink went wrong. The result was a `throws Exception`, which also means that you have to catch `Exception` which you often don't want (because this included `RuntimeException` and you typically want runtime exception to bubble up a bit further, since they denote bugs by encouraged design).
    
    The only place where `throws Exception` really makes sense to me is for `MapFunction` and the likes, to allow them to propagate any type of exception and let recovery handle them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3368: [FLINK-5854] [core] Add base Flink Exception classes

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the issue:

    https://github.com/apache/flink/pull/3368
  
    Big +1 from my side. I think that more specific exceptions helps to make people think more about the cause of an exception and, thus, also how it should be handled. Especially the fact that we always catch `RuntimeException` when having `throws Exception` in the method definition annoyed me a lot.
    
    @uce I think we should not per se wrap all current `Exceptions` into a `FlinkRuntimeException`. Better to do a case distinction here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3368: [FLINK-5854] [core] Add base Flink Exception classes

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/3368
  
    Thanks for the comments. Will address the issues and remove the "no message, no cause" constructors. We should not encourage exceptions without information.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3368: [FLINK-5854] [core] Add base Flink Exception class...

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

    https://github.com/apache/flink/pull/3368#discussion_r102072425
  
    --- Diff: flink-core/src/main/java/org/apache/flink/util/DynamicCodeLoadingException.java ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.flink.util;
    +
    +import org.apache.flink.annotation.Public;
    +
    +/**
    + * An exception that is thrown if the dynamic instantiation of code fails.
    + * 
    + * <p>This exception is supposed to "sum up" the zoo of exceptions typically thrown around
    + * dynamic code loading and instantiations:
    + * 
    + * <pre>{@code
    + * try {
    + *     Class.forName(classname).asSubclass(TheType.class).newInstance();
    + * }
    + * catch (ClassNotFoundException | ClassCastException | InstantiationException | IllegalAccessException e) {
    + *     throw new DynamicCodeLoadingException(e);
    + * }
    + * }</pre>
    + */
    +@Public
    +public class DynamicCodeLoadingException extends FlinkException {
    +
    +	private static final long serialVersionUID = -25138443817255490L;
    +
    +	/**
    +	 * Creates a new exception with the given message and cause
    +	 *
    +	 * @param message The exception message
    +	 * @param cause The exception that caused this exception
    +	 */
    +	public DynamicCodeLoadingException(String message, Throwable cause) {
    --- End diff --
    
    Would it make sense to make this constructor more explicit in the type of exceptions it accepts? (i.e. one constructor each for the exceptions that are typically thrown in situations that we want to cover)
    
    It's probably just be bloat, but maybe it would prevent misuse of this exception.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3368: [FLINK-5854] [core] Add base Flink Exception class...

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

    https://github.com/apache/flink/pull/3368#discussion_r102071979
  
    --- Diff: flink-core/src/main/java/org/apache/flink/util/DynamicCodeLoadingException.java ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.flink.util;
    +
    +import org.apache.flink.annotation.Public;
    +
    +/**
    + * An exception that is thrown if the dynamic instantiation of code fails.
    + * 
    + * <p>This exception is supposed to "sum up" the zoo of exceptions typically thrown around
    + * dynamic code loading and instantiations:
    + * 
    + * <pre>{@code
    + * try {
    + *     Class.forName(classname).asSubclass(TheType.class).newInstance();
    + * }
    + * catch (ClassNotFoundException | ClassCastException | InstantiationException | IllegalAccessException e) {
    + *     throw new DynamicCodeLoadingException(e);
    --- End diff --
    
    there is no constructor that matches this line of the javadoc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3368: [FLINK-5854] [core] Add base Flink Exception classes

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/3368
  
    It is arguable whether exceptions should ever have a constructor without a message, I simply did that here for convenience. I have no strong feelings about removing the zero argument constructors.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3368: [FLINK-5854] [core] Add base Flink Exception classes

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/3368
  
    Any reservations against merging this after addressing the comments above?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3368: [FLINK-5854] [core] Add base Flink Exception class...

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

    https://github.com/apache/flink/pull/3368#discussion_r102430926
  
    --- Diff: flink-core/src/main/java/org/apache/flink/util/DynamicCodeLoadingException.java ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.flink.util;
    +
    +import org.apache.flink.annotation.Public;
    +
    +/**
    + * An exception that is thrown if the dynamic instantiation of code fails.
    + * 
    + * <p>This exception is supposed to "sum up" the zoo of exceptions typically thrown around
    + * dynamic code loading and instantiations:
    + * 
    + * <pre>{@code
    + * try {
    + *     Class.forName(classname).asSubclass(TheType.class).newInstance();
    + * }
    + * catch (ClassNotFoundException | ClassCastException | InstantiationException | IllegalAccessException e) {
    + *     throw new DynamicCodeLoadingException(e);
    --- End diff --
    
    Agreed, will fix this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3368: [FLINK-5854] [core] Add base Flink Exception classes

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on the issue:

    https://github.com/apache/flink/pull/3368
  
    This looks good, but what is the reason behind the empty constructors that create an exception without message or cause?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3368: [FLINK-5854] [core] Add base Flink Exception class...

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

    https://github.com/apache/flink/pull/3368#discussion_r102072659
  
    --- Diff: flink-core/src/main/java/org/apache/flink/util/FlinkRuntimeException.java ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.flink.util;
    +
    +import org.apache.flink.annotation.Public;
    +
    +/**
    + * Base class of all Flink-specific unchecked exceptions.
    + */
    +@Public
    +public class FlinkRuntimeException extends RuntimeException {
    +
    +	private static final long serialVersionUID = 193141189399279147L;
    +
    +	/**
    +	 * Creates a new exception with a null message and null cause. 
    +	 */
    +	public FlinkRuntimeException() {
    --- End diff --
    
    Is there a reasonable use-case for an exception without an error message or cause?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---