[ https://issues.apache.org/jira/browse/FLINK-6655?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16064802#comment-16064802 ]
ASF GitHub Bot commented on FLINK-6655: --------------------------------------- Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/4156#discussion_r124266875 --- Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala --- @@ -255,4 +255,47 @@ class MemoryArchivist( graphs.remove(jobID) } } + + /** + * Checks and normalizes the archive path URI. This method first checks the validity of the + * URI (scheme, path, availability of a matching file system) and then normalizes the URL + * to a path. + * + * If the URI does not include an authority, but the file system configured for the URI has an + * authority, then the normalized path will include this authority. + * + * @param archivePathUri The URI to check and normalize. + * @return a normalized URI as a Path. + * + * @throws IllegalArgumentException Thrown, if the URI misses schema or path. + * @throws IOException Thrown, if no file system can be found for the URI's scheme. + */ + @throws[IOException] + private def validateAndNormalizeUri(archivePathUri: URI): Path = { + val scheme = archivePathUri.getScheme + val path = archivePathUri.getPath + + // some validity checks + if (scheme == null) { + throw new IllegalArgumentException("The scheme (hdfs://, file://, etc) is null. " + + "Please specify the file system scheme explicitly in the URI: " + archivePathUri) + } + + if (path == null) { + throw new IllegalArgumentException("The path to store the job archives is null. " + + "Please specify a directory path for storing job archives. and the URI is: " + archivePathUri) + } + + if (path.length == 0 || path == "/") { + throw new IllegalArgumentException("Cannot use the root directory for storing job archives.") + } + + if (!FileSystem.isFlinkSupportedScheme(archivePathUri.getScheme)) { + // skip verification checks for non-flink supported filesystem + // this is because the required filesystem classes may not be available to the flink client + throw new IllegalArgumentException("Cannot use the " + archivePathUri.getScheme + " scheme, only hdfs, " + --- End diff -- This is bound to be outdated at some point, so tet's re-use an exception from the FileSystem class: ``` "No file system found with scheme " + scheme + ", referenced in file URI '" + archivePathUri.toString() + "'." ``` > Misleading error message when HistoryServer path is empty > --------------------------------------------------------- > > Key: FLINK-6655 > URL: https://issues.apache.org/jira/browse/FLINK-6655 > Project: Flink > Issue Type: Bug > Components: History Server > Affects Versions: 1.3.0 > Reporter: Timo Walther > Assignee: mingleizhang > Priority: Minor > > If the HistoryServer {{jobmanager.archive.fs.dir}} if e.g. {{file://}}. The > following exception mentions checkpoints, which is misleading. > {code} > java.lang.IllegalArgumentException: Cannot use the root directory for > checkpoints. > at > org.apache.flink.runtime.state.filesystem.FsStateBackend.validateAndNormalizeUri(FsStateBackend.java:358) > at > org.apache.flink.runtime.jobmanager.MemoryArchivist.org$apache$flink$runtime$jobmanager$MemoryArchivist$$archiveJsonFiles(MemoryArchivist.scala:201) > at > org.apache.flink.runtime.jobmanager.MemoryArchivist$$anonfun$handleMessage$1.applyOrElse(MemoryArchivist.scala:107) > at > scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33) > at > scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33) > at > scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25) > at > org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:33) > at > org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:28) > at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:118) > at > org.apache.flink.runtime.LogMessages$$anon$1.applyOrElse(LogMessages.scala:28) > at akka.actor.Actor$class.aroundReceive(Actor.scala:467) > at > org.apache.flink.runtime.jobmanager.MemoryArchivist.aroundReceive(MemoryArchivist.scala:65) > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516) > at akka.actor.ActorCell.invoke(ActorCell.scala:487) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:238) > at akka.dispatch.Mailbox.run(Mailbox.scala:220) > at > akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397) > at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) > at > scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) > at > scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) > at > scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) > {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)