[jira] Commented: (HADOOP-2178) Job history on HDFS
[ https://issues.apache.org/jira/browse/HADOOP-2178?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12558712#action_12558712 ] eric baldeschwieler commented on HADOOP-2178: - The user vs admin dimension is not really the same as the static vs HOD dimension. Even in a static JT, the job history is probably usefully part of the output directory (for a user). I think this should just be part of the output API, no matter how the cluster is configured. This will be much easier to document and use. Then we could handle static job trackers as Amareshwari describes. In the case of HOD deployed JTs, I think we can then either set hadoop.job.history.location to NULL or to a HOD specified output directory, probably on HDFS or another shared FS. This would be useful if the user is running a lot jobs through a single HOD instance. NULL is probably a fine default. --- I agree with runping that we need to define an API for collecting central stats from HOD deployed JTs. I think a configured URL is ok as an API, but we need to be clear that this output will be for central collection, not user diagnostics and as such the layout should be optimized to simplify that (probably time sorted, not user sorted for example). Ideally this could be a single file per JT instance. Job history on HDFS --- Key: HADOOP-2178 URL: https://issues.apache.org/jira/browse/HADOOP-2178 Project: Hadoop Issue Type: Improvement Components: mapred Reporter: Amareshwari Sri Ramadasu Assignee: Amareshwari Sri Ramadasu Fix For: 0.16.0 This issue addresses the following items : 1. Check for accuracy of job tracker history logs. 2. After completion of the job, copy the JobHistory.log(Master index file) and the job history files to the DFS. 3. User can load the history with commands bin/hadoop job -history directory or bin/hadoop job -history jobid This will start a stand-alone jetty and load jsps -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2066) filenames with ':' colon throws java.lang.IllegalArgumentException
[ https://issues.apache.org/jira/browse/HADOOP-2066?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12558928#action_12558928 ] eric baldeschwieler commented on HADOOP-2066: - Good idea. Something like this would address my concerns. filenames with ':' colon throws java.lang.IllegalArgumentException -- Key: HADOOP-2066 URL: https://issues.apache.org/jira/browse/HADOOP-2066 Project: Hadoop Issue Type: Bug Components: dfs Reporter: lohit vijayarenu Attachments: 2066_20071022.patch, HADOOP-2066.patch File names containing colon : throws java.lang.IllegalArgumentException while LINUX file system supports it. $ hadoop dfs -put ./testfile-2007-09-24-03:00:00.gz filenametest Exception in thread main java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: testfile-2007-09-24-03:00:00.gz at org.apache.hadoop.fs.Path.initialize(Path.java:140) at org.apache.hadoop.fs.Path.init(Path.java:126) at org.apache.hadoop.fs.Path.init(Path.java:50) at org.apache.hadoop.fs.FileUtil.checkDest(FileUtil.java:273) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:117) at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:776) at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:757) at org.apache.hadoop.fs.FsShell.copyFromLocal(FsShell.java:116) at org.apache.hadoop.fs.FsShell.run(FsShell.java:1229) at org.apache.hadoop.util.ToolBase.doMain(ToolBase.java:187) at org.apache.hadoop.fs.FsShell.main(FsShell.java:1342) Caused by: java.net.URISyntaxException: Relative path in absolute URI: testfile-2007-09-24-03:00:00.gz at java.net.URI.checkPath(URI.java:1787) at java.net.URI.init(URI.java:735) at org.apache.hadoop.fs.Path.initialize(Path.java:137) ... 10 more Path(String pathString) when given a filename which contains ':' treats it as URI and selects anything before ':' as scheme, which in this case is clearly not a valid scheme. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2559) DFS should place one replica per rack
[ https://issues.apache.org/jira/browse/HADOOP-2559?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12558459#action_12558459 ] eric baldeschwieler commented on HADOOP-2559: - A further issue is that as discussed on the list, we get very uneven distribution of data on the cluster when you have a small number of clients writing a lot of data. A preference for one copy on the same rack unless that rack is substantially more full than most does make sense, but a preference for the same node seems problematic. Likewise, the choice of putting two blocks on the source rack seems to lead to a lot of imbalance. We could get the same bandwidth reduction by putting 2 copies on the second rack if it has more free space than the source rack. We could also choose the not allow two copies on a rack in the standard 3 replica case, but that is a separable issue. DFS should place one replica per rack - Key: HADOOP-2559 URL: https://issues.apache.org/jira/browse/HADOOP-2559 Project: Hadoop Issue Type: Improvement Components: dfs Reporter: Runping Qi Currently, when writing out a block, dfs will place one copy to a local data node, one copy to a rack local node and another one to a remote node. This leads to a number of undesired properties: 1. The block will be rack-local to two tacks instead of three, reducing the advantage of rack locality based scheduling by 1/3. 2. The Blocks of a file (especiallya large file) are unevenly distributed over the nodes: One third will be on the local node, and two thirds on the nodes on the same rack. This may make some nodes full much faster than others, increasing the need of rebalancing. Furthermore, this also make some nodes become hot spots if those big files are popular and accessed by many applications. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2066) filenames with ':' colon throws java.lang.IllegalArgumentException
[ https://issues.apache.org/jira/browse/HADOOP-2066?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12558464#action_12558464 ] eric baldeschwieler commented on HADOOP-2066: - I think it makes great sense to support a URI interface to the FS. I recall championing that. I'm not at all sure it makes sense to define what is a valid filename based on a URI library. URI can support quoting and we will undoubtedly support several non-URI interfaces to the FS. I can create filenames with colon's on my mac btw. Not that I see why our filenames should be defined by the limits of what all supported client platforms can support. I think this issue requires more thought. filenames with ':' colon throws java.lang.IllegalArgumentException -- Key: HADOOP-2066 URL: https://issues.apache.org/jira/browse/HADOOP-2066 Project: Hadoop Issue Type: Bug Components: dfs Reporter: lohit vijayarenu Attachments: 2066_20071022.patch, HADOOP-2066.patch File names containing colon : throws java.lang.IllegalArgumentException while LINUX file system supports it. $ hadoop dfs -put ./testfile-2007-09-24-03:00:00.gz filenametest Exception in thread main java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: testfile-2007-09-24-03:00:00.gz at org.apache.hadoop.fs.Path.initialize(Path.java:140) at org.apache.hadoop.fs.Path.init(Path.java:126) at org.apache.hadoop.fs.Path.init(Path.java:50) at org.apache.hadoop.fs.FileUtil.checkDest(FileUtil.java:273) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:117) at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:776) at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:757) at org.apache.hadoop.fs.FsShell.copyFromLocal(FsShell.java:116) at org.apache.hadoop.fs.FsShell.run(FsShell.java:1229) at org.apache.hadoop.util.ToolBase.doMain(ToolBase.java:187) at org.apache.hadoop.fs.FsShell.main(FsShell.java:1342) Caused by: java.net.URISyntaxException: Relative path in absolute URI: testfile-2007-09-24-03:00:00.gz at java.net.URI.checkPath(URI.java:1787) at java.net.URI.init(URI.java:735) at org.apache.hadoop.fs.Path.initialize(Path.java:137) ... 10 more Path(String pathString) when given a filename which contains ':' treats it as URI and selects anything before ':' as scheme, which in this case is clearly not a valid scheme. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2014) Job Tracker should not clobber the data locality of tasks
[ https://issues.apache.org/jira/browse/HADOOP-2014?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12558155#action_12558155 ] eric baldeschwieler commented on HADOOP-2014: - An ideal solution would maintain some sort of prioritized list of maps / node / rack so that we execute work first that is unlikely to find another efficient location to execute. It would also make sense to place some no local work early, since these tasks run slowly, on nodes that are likely to run out of local work relatively early. One could also pay attention to IO load on each source node... At a minimum we should track maps that have no local option and schedule them first when a node has no local option. (As doug cutting suggested in hadoop-2560 Job Tracker should not clobber the data locality of tasks - Key: HADOOP-2014 URL: https://issues.apache.org/jira/browse/HADOOP-2014 Project: Hadoop Issue Type: Bug Components: mapred Reporter: Runping Qi Assignee: Devaraj Das Currently, when the Job Tracker assigns a mapper task to a task tracker and there is no local split to the task tracker, the job tracker will find the first runable task in the mast task list and assign the task to the task tracker. The split for the task is not local to the task tracker, of course. However, the split may be local to other task trackers. Assigning the that task, to that task tracker may decrease the potential number of mapper attempts with data locality. The desired behavior in this situation is to choose a task whose split is not local to any task tracker. Resort to the current behavior only if no such task is found. In general, it will be useful to know the number of task trackers to which each split is local. To assign a task to a task tracker, the job tracker should first try to pick a task that is local to the task tracker and that has minimal number of task trackers to which it is local. If no task is local to the task tracker, the job tracker should try to pick a task that has minimal number of task trackers to which it is local. It is worthwhile to instrument the job tracker code to report the number of splits that are local to some task trackers. That should be the maximum number of tasks with data locality. By comparing that number with the the actual number of data local mappers launched, we can know the effectiveness of the job tracker scheduling. When we introduce rack locality, we should apply the same principle. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Issue Comment Edited: (HADOOP-2014) Job Tracker should not clobber the data locality of tasks
[ https://issues.apache.org/jira/browse/HADOOP-2014?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12558155#action_12558155 ] eric14 edited comment on HADOOP-2014 at 1/11/08 2:41 PM: -- An ideal solution would maintain some sort of prioritized list of maps / node / rack so that we execute work first that is unlikely to find another efficient location to execute. It would also make sense to place some no local work early, since these tasks run slowly, on nodes that are likely to run out of local work relatively early. One could also pay attention to IO load on each source node... At a minimum we should track maps that have no local option and schedule them first when a node has no local option. (As doug cutting suggested in HADOOP-2560) was (Author: eric14): An ideal solution would maintain some sort of prioritized list of maps / node / rack so that we execute work first that is unlikely to find another efficient location to execute. It would also make sense to place some no local work early, since these tasks run slowly, on nodes that are likely to run out of local work relatively early. One could also pay attention to IO load on each source node... At a minimum we should track maps that have no local option and schedule them first when a node has no local option. (As doug cutting suggested in hadoop-2560 Job Tracker should not clobber the data locality of tasks - Key: HADOOP-2014 URL: https://issues.apache.org/jira/browse/HADOOP-2014 Project: Hadoop Issue Type: Bug Components: mapred Reporter: Runping Qi Assignee: Devaraj Das Currently, when the Job Tracker assigns a mapper task to a task tracker and there is no local split to the task tracker, the job tracker will find the first runable task in the mast task list and assign the task to the task tracker. The split for the task is not local to the task tracker, of course. However, the split may be local to other task trackers. Assigning the that task, to that task tracker may decrease the potential number of mapper attempts with data locality. The desired behavior in this situation is to choose a task whose split is not local to any task tracker. Resort to the current behavior only if no such task is found. In general, it will be useful to know the number of task trackers to which each split is local. To assign a task to a task tracker, the job tracker should first try to pick a task that is local to the task tracker and that has minimal number of task trackers to which it is local. If no task is local to the task tracker, the job tracker should try to pick a task that has minimal number of task trackers to which it is local. It is worthwhile to instrument the job tracker code to report the number of splits that are local to some task trackers. That should be the maximum number of tasks with data locality. By comparing that number with the the actual number of data local mappers launched, we can know the effectiveness of the job tracker scheduling. When we introduce rack locality, we should apply the same principle. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2560) Combining multiple input blocks into one mapper
[ https://issues.apache.org/jira/browse/HADOOP-2560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12557812#action_12557812 ] eric baldeschwieler commented on HADOOP-2560: - We queue each map for each candidate node (and now presumably rack) and pull them from consideration once they are scheduled on any node. This gets much more complicated with map sets, since you will need to tag which maps in one set have been executed somewhere else and then replace them... Much simpler to make the late binding decision to bundle them. I get a feeling this issue will be revisit more than once... Combining multiple input blocks into one mapper --- Key: HADOOP-2560 URL: https://issues.apache.org/jira/browse/HADOOP-2560 Project: Hadoop Issue Type: Bug Reporter: Runping Qi Currently, an input split contains a consecutive chunk of input file, which by default, corresponding to a DFS block. This may lead to a large number of mapper tasks if the input data is large. This leads to the following problems: 1. Shuffling cost: since the framework has to move M * R map output segments to the nodes running reducers, larger M means larger shuffling cost. 2. High JVM initialization overhead 3. Disk fragmentation: larger number of map output files means lower read throughput for accessing them. Ideally, you want to keep the number of mappers to no more than 16 times the number of nodes in the cluster. To achive that, we can increase the input split size. However, if a split span over more than one dfs block, you lose the data locality scheduling benefits. One way to address this problem is to combine multiple input blocks with the same rack into one split. If in average we combine B blocks into one split, then we will reduce the number of mappers by a factor of B. Since all the blocks for one mapper share a rack, thus we can benefit from rack-aware scheduling. Thoughts? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2560) Combining multiple input blocks into one mapper
[ https://issues.apache.org/jira/browse/HADOOP-2560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12557396#action_12557396 ] eric baldeschwieler commented on HADOOP-2560: - A) this is important because it could lead to big throughput gains and seek reductions. B) It is not going to work to combine splits statically because block replicas are not co-resident. This would lead to a huge performance hit due to loss of locality. I think we will need to invest in more complexity to get the desired performance improvement here. My gut is that we should do this dynamically in the task trackers. This would let us do it when we are seeing good io throughput. The map driver could always just request a new split after each input finishes. The TT would keep a small number of candidate splits locally and decide after each map completes a split if it is going to hand it another one. None of the public interfaces would need to change. We would need to change the JT quite a bit to manage maps publishing split collections, but it seems fairly straightforward. We could realize a huge performance gain on simple scanning jobs that process input quickly. We could also see good shuffle improvements. This would interact with speculative execution in undesirable ways... Something to watch-out for. There are a whole class of collation optimizations here. The fact that we are sorting early may make a lot of them harder... ugh. A related idea runping and I discussed is that if you have multiple spills in a map (combined or unchanged map), there is no point collating the spills if the reduce partitions are relatively large (say 1MB). We could just make each spill an output to the reduces. Even if they are small it would be more efficient to collate in larger units than within a single map, but that starts really broadening the design space... Could static splits combinations work at all? Yes I think they might if we produced only a small number and executed them early, but this would reduce the possible gain we could get. Combining multiple input blocks into one mapper --- Key: HADOOP-2560 URL: https://issues.apache.org/jira/browse/HADOOP-2560 Project: Hadoop Issue Type: Bug Reporter: Runping Qi Currently, an input split contains a consecutive chunk of input file, which by default, corresponding to a DFS block. This may lead to a large number of mapper tasks if the input data is large. This leads to the following problems: 1. Shuffling cost: since the framework has to move M * R map output segments to the nodes running reducers, larger M means larger shuffling cost. 2. High JVM initialization overhead 3. Disk fragmentation: larger number of map output files means lower read throughput for accessing them. Ideally, you want to keep the number of mappers to no more than 16 times the number of nodes in the cluster. To achive that, we can increase the input split size. However, if a split span over more than one dfs block, you lose the data locality scheduling benefits. One way to address this problem is to combine multiple input blocks with the same rack into one split. If in average we combine B blocks into one split, then we will reduce the number of mappers by a factor of B. Since all the blocks for one mapper share a rack, thus we can benefit from rack-aware scheduling. Thoughts? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2560) Combining multiple input blocks into one mapper
[ https://issues.apache.org/jira/browse/HADOOP-2560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12557477#action_12557477 ] eric baldeschwieler commented on HADOOP-2560: - Nodes operate at different rates. Failures happen. In the face of several jobs running, some nodes may not even become available in a timely manner. I think a static approach will not allow both the performance gains desired and preservation of reasonable throughput. The current system takes full advantage of mapping jobs to nodes dynamically. A static combination of splits will break all of this. One could perhaps do something like what you suggest dynamically in the JT when a TT requests a new job. This might be a good compromise implementation. This would also let you observe some global statistics on speed of maps size of outputs which would let you optimize cluster sizes. Of course doing this all dynamically on the TTs might use fewer JT resources. Combining multiple input blocks into one mapper --- Key: HADOOP-2560 URL: https://issues.apache.org/jira/browse/HADOOP-2560 Project: Hadoop Issue Type: Bug Reporter: Runping Qi Currently, an input split contains a consecutive chunk of input file, which by default, corresponding to a DFS block. This may lead to a large number of mapper tasks if the input data is large. This leads to the following problems: 1. Shuffling cost: since the framework has to move M * R map output segments to the nodes running reducers, larger M means larger shuffling cost. 2. High JVM initialization overhead 3. Disk fragmentation: larger number of map output files means lower read throughput for accessing them. Ideally, you want to keep the number of mappers to no more than 16 times the number of nodes in the cluster. To achive that, we can increase the input split size. However, if a split span over more than one dfs block, you lose the data locality scheduling benefits. One way to address this problem is to combine multiple input blocks with the same rack into one split. If in average we combine B blocks into one split, then we will reduce the number of mappers by a factor of B. Since all the blocks for one mapper share a rack, thus we can benefit from rack-aware scheduling. Thoughts? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2178) Job history on HDFS
[ https://issues.apache.org/jira/browse/HADOOP-2178?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12556907#action_12556907 ] eric baldeschwieler commented on HADOOP-2178: - I don't think it makes sense to address HADOOP-1876 by storing history on HDFS by default. I think it is better to keep the server logs on the local machine by default and allow a configuration to store them in a well specified place on HDFS. This will keep a HOD user from creating leavings all over the HDFS, but allow for alternate uses like HADOOP-1876. We have 3 use cases we need to think about separately: 1) A cluster with one permanent JT. This should have a single well know place it logs. 2) A HOD JT, this should send logs to a user specified directory 3) Central logging of job summary data that works in either of the above cases. We should create a distinct JIRA to discuss this. Job history on HDFS --- Key: HADOOP-2178 URL: https://issues.apache.org/jira/browse/HADOOP-2178 Project: Hadoop Issue Type: Improvement Components: mapred Reporter: Amareshwari Sri Ramadasu Assignee: Amareshwari Sri Ramadasu Fix For: 0.16.0 This issue addresses the following items : 1. Check for accuracy of job tracker history logs. 2. After completion of the job, copy the JobHistory.log(Master index file) and the job history files to the DFS. 3. User can load the history with commands bin/hadoop job -history directory or bin/hadoop job -history jobid This will start a stand-alone jetty and load jsps -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (HADOOP-2497) sequence file does not detect corruption in key/value lengths
sequence file does not detect corruption in key/value lengths - Key: HADOOP-2497 URL: https://issues.apache.org/jira/browse/HADOOP-2497 Project: Hadoop Issue Type: Bug Reporter: eric baldeschwieler From: [EMAIL PROTECTED] Subject: Re: question on Hadoop configuration for non cpu intensive jobs - 0.15.1 Date: December 28, 2007 7:20:11 AM PST To: [EMAIL PROTECTED] Our OOM was being caused by a damaged sequence data file. We had assumed that the sequence files had checksums, which appears to be in correct. The deserializer was reading a bad length out of the file and trying to allocate 4gig of ram. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (HADOOP-2491) generalize the TT / JT servers to handle more generic tasks
generalize the TT / JT servers to handle more generic tasks --- Key: HADOOP-2491 URL: https://issues.apache.org/jira/browse/HADOOP-2491 Project: Hadoop Issue Type: Improvement Components: mapred Reporter: eric baldeschwieler We've been discussing a proposal to generalize the TT / JT servers to handle more generic tasks and move job specific work out of the job tracker and into client code so the whole system is both much more general and has more coherent layering. The result would look more like condor/pbs like systems (or presumably borg) with map-reduce as a user job. Such a system would allow the current map-reduce code to coexist with other work-queuing libraries or maybe even persistent services on the same Hadoop cluster, although that would be a stretch goal. We'll kick off a thread with some documents soon. Our primary goal in going this way would be to get better utilization out of map-reduce clusters and support a richer scheduling model. The ability to support alternative job frameworks would just be gravy! Putting this in as a place holder. Hope to get folks talking about this to post some more detail. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1298) adding user info to file
[ https://issues.apache.org/jira/browse/HADOOP-1298?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12553689 ] eric baldeschwieler commented on HADOOP-1298: - Agreed. we don't want to see a regression in NN performance. That would be a blocking issue for any release in which it occurs. We have to do performance validation of changes like this. adding user info to file Key: HADOOP-1298 URL: https://issues.apache.org/jira/browse/HADOOP-1298 Project: Hadoop Issue Type: New Feature Components: dfs, fs Reporter: Kurtis Heimerl Assignee: Christophe Taton Attachments: 1298_2007-09-22_1.patch, 1298_2007-10-04_1.patch, 1298_20071219c.patch, hadoop-user-munncha.patch17, HDFSPermissionSpecification4.pdf I'm working on adding a permissions model to hadoop's DFS. The first step is this change, which associates user info with files. Following this I'll assoicate permissions info, then block methods based on that user info, then authorization of the user info. So, right now i've implemented adding user info to files. I'm looking for feedback before I clean this up and make it offical. I wasn't sure what release, i'm working off trunk. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (HADOOP-2429) The lowest level map-reduce APIs should be byte oriented
The lowest level map-reduce APIs should be byte oriented Key: HADOOP-2429 URL: https://issues.apache.org/jira/browse/HADOOP-2429 Project: Hadoop Issue Type: Improvement Reporter: eric baldeschwieler As discussed here: https://issues.apache.org/jira/browse/HADOOP-1986#action_12551237 The templates, serializers and other complexities that allow map-reduce to use arbitrary types complicate the design and lead to lots of object creates and other overhead that a byte oriented design would not suffer. I believe the lowest level implementation of hadoop map-reduce should have byte string oriented APIs (for keys and values). This API would be more performant, simpler and more easily cross language. The existing API could be maintained as a thin layer on top of the leaner API. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1986) Add support for a general serialization mechanism for Map Reduce
[ https://issues.apache.org/jira/browse/HADOOP-1986?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12551858 ] eric baldeschwieler commented on HADOOP-1986: - Interesting point - it sounds like such a design might be achieved by moving the serialization stuff into its own layer above core Map Reduce. EXACTLY! I've created HADOOP-2429 to discuss this point. Add support for a general serialization mechanism for Map Reduce Key: HADOOP-1986 URL: https://issues.apache.org/jira/browse/HADOOP-1986 Project: Hadoop Issue Type: New Feature Components: mapred Reporter: Tom White Assignee: Tom White Fix For: 0.16.0 Attachments: hadoop-serializer-v2.tar.gz, SerializableWritable.java, serializer-v1.patch, serializer-v2.patch Currently Map Reduce programs have to use WritableComparable-Writable key-value pairs. While it's possible to write Writable wrappers for other serialization frameworks (such as Thrift), this is not very convenient: it would be nicer to be able to use arbitrary types directly, without explicit wrapping and unwrapping. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1986) Add support for a general serialization mechanism for Map Reduce
[ https://issues.apache.org/jira/browse/HADOOP-1986?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12551237 ] eric baldeschwieler commented on HADOOP-1986: - The code seems more readable on average after this work, so i think it represents a step forward from where we were. Good stuff tom. That said... I just went through this whole thread. I find it quite mind bending. I remain convinced that the use of templated types at all in our lowest level API is a mistake and all of this templated foo should be an optional layer on top of a basic byte oriented API. Other famous map-reduce implementations make that choice (byte API only) and they get to bypass all of this discussion and complexity. Something I ask everyone to think about. That is not a reason to reject this patch, since I think this looks like an incremental improvement. The code doesn't look like it adds any appreciable overhead, a function call and a comparison per read seems likely to be in the noise, unless the stream class bites us. Have you validated that we're not taking on extra cost here? We're trying to strip out layers of copies and other framework fat. Add support for a general serialization mechanism for Map Reduce Key: HADOOP-1986 URL: https://issues.apache.org/jira/browse/HADOOP-1986 Project: Hadoop Issue Type: New Feature Components: mapred Reporter: Tom White Assignee: Tom White Fix For: 0.16.0 Attachments: hadoop-serializer-v2.tar.gz, SerializableWritable.java, serializer-v1.patch, serializer-v2.patch Currently Map Reduce programs have to use WritableComparable-Writable key-value pairs. While it's possible to write Writable wrappers for other serialization frameworks (such as Thrift), this is not very convenient: it would be nicer to be able to use arbitrary types directly, without explicit wrapping and unwrapping. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2012) Periodic verification at the Datanode
[ https://issues.apache.org/jira/browse/HADOOP-2012?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12548756 ] eric baldeschwieler commented on HADOOP-2012: - I'd really rather see us get this right for supported platforms rather than declare an issue done when we know it does not work on some platforms. This is particularly vexing when design choices could clearly be made that would avoid these issues. -1 Periodic verification at the Datanode - Key: HADOOP-2012 URL: https://issues.apache.org/jira/browse/HADOOP-2012 Project: Hadoop Issue Type: New Feature Components: dfs Reporter: Raghu Angadi Assignee: Raghu Angadi Fix For: 0.16.0 Attachments: HADOOP-2012.patch, HADOOP-2012.patch, HADOOP-2012.patch, HADOOP-2012.patch Currently on-disk data corruption on data blocks is detected only when it is read by the client or by another datanode. These errors are detected much earlier if datanode can periodically verify the data checksums for the local blocks. Some of the issues to consider : - How should we check the blocks ( no more often than once every couple of weeks ?) - How do we keep track of when a block was last verfied ( there is a .meta file associcated with each lock ). - What action to take once a corruption is detected - Scanning should be done as a very low priority with rest of the datanode disk traffic in mind. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2154) Non-interleaved checksums would optimize block transfers.
[ https://issues.apache.org/jira/browse/HADOOP-2154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12546800 ] eric baldeschwieler commented on HADOOP-2154: - Still +1, but apologies. I was thinking about the read case when I wrote the comment, I think what raghu stated makes more sense without my additions. Non-interleaved checksums would optimize block transfers. - Key: HADOOP-2154 URL: https://issues.apache.org/jira/browse/HADOOP-2154 Project: Hadoop Issue Type: Improvement Components: dfs Affects Versions: 0.14.0 Reporter: Konstantin Shvachko Assignee: Rajagopal Natarajan Fix For: 0.16.0 Currently when a block is transfered to a data-node the client interleaves data chunks with the respective checksums. This requires creating an extra copy of the original data in a new buffer interleaved with the crcs. We can avoid extra copying if the data and the crc are fed to the socket one after another. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2154) Non-interleaved checksums would optimize block transfers.
[ https://issues.apache.org/jira/browse/HADOOP-2154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12546587 ] eric baldeschwieler commented on HADOOP-2154: - +1 We should not be doing these copies and interleaves if we can avoid them. A lot of change here, but if we could move to a protocol where the client requests a buffer of bytes, rather than just pushing bytes, we could start the response with a CRCs list, followed by the bytes. This would require less RAM on the client side (I think). Can we just memory map the block and then copy the requested chunk it directly to the socket or use other tricks to reduce copies further? (I'm NIO naive) Non-interleaved checksums would optimize block transfers. - Key: HADOOP-2154 URL: https://issues.apache.org/jira/browse/HADOOP-2154 Project: Hadoop Issue Type: Improvement Components: dfs Affects Versions: 0.14.0 Reporter: Konstantin Shvachko Assignee: Rajagopal Natarajan Fix For: 0.16.0 Currently when a block is transfered to a data-node the client interleaves data chunks with the respective checksums. This requires creating an extra copy of the original data in a new buffer interleaved with the crcs. We can avoid extra copying if the data and the crc are fed to the socket one after another. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1722) Make streaming to handle non-utf8 byte array
[ https://issues.apache.org/jira/browse/HADOOP-1722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12542991 ] eric baldeschwieler commented on HADOOP-1722: - I think arkady's point is much more to the point than this quoting proposal, which I think is going the wrong way! There are two interfaces here - that between man reduce and that into map and out of reduce. I think they deserve different handling. 1) map in reduce out - Should by default just consume bytes and produce bytes. The framework should do no interpretation or quoting. It should not try to break the output into lines, keys values, etc. It is just a byte stream. This will allow true binary output with zero hassle. Some thought on splits is clearly needed... 2) map out reduce in - Here we clearly need keys and values. But i think quoting might be the wrong direction. It should certainly not be the default. I think we should consider just providing an option that specifies a new binary format will be used. here. Maybe a 4 byte length followed a binary key followed by a 4 byte length and then a binary value? Maybe with a record terminator for sanity checking? Two observations: 1) Adding quoting by default will break all kinds of programs that work with streaming today. This is undesirable. We should add an option, not change the default behavior. 2) Streaming should not use utf8 anywhere! It should assume that it is processing a stream of bytes that contains certain signal bytes '\n' and '\t'. I think we all agree on this. treating the byte stream as a character stream just confuses things. Make streaming to handle non-utf8 byte array Key: HADOOP-1722 URL: https://issues.apache.org/jira/browse/HADOOP-1722 Project: Hadoop Issue Type: Improvement Components: contrib/streaming Reporter: Runping Qi Assignee: Christopher Zimmerman Right now, the streaming framework expects the output sof the steam process (mapper or reducer) are line oriented UTF-8 text. This limit makes it impossible to use those programs whose outputs may be non-UTF-8 (international encoding, or maybe even binary data). Streaming can overcome this limit by introducing a simple encoding protocol. For example, it can allow the mapper/reducer to hexencode its keys/values, the framework decodes them in the Java side. This way, as long as the mapper/reducer executables follow this encoding protocol, they can output arabitary bytearray and the streaming framework can handle them. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (HADOOP-1298) adding user info to file
[ https://issues.apache.org/jira/browse/HADOOP-1298?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] eric baldeschwieler updated HADOOP-1298: Comment: was deleted adding user info to file Key: HADOOP-1298 URL: https://issues.apache.org/jira/browse/HADOOP-1298 Project: Hadoop Issue Type: New Feature Components: dfs, fs Reporter: Kurtis Heimerl Assignee: Christophe Taton Attachments: 1298_2007-09-22_1.patch, 1298_2007-10-04_1.patch, hadoop-user-munncha.patch17 I'm working on adding a permissions model to hadoop's DFS. The first step is this change, which associates user info with files. Following this I'll assoicate permissions info, then block methods based on that user info, then authorization of the user info. So, right now i've implemented adding user info to files. I'm looking for feedback before I clean this up and make it offical. I wasn't sure what release, i'm working off trunk. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2144) Data node process consumes 180% cpu
[ https://issues.apache.org/jira/browse/HADOOP-2144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12539752 ] eric baldeschwieler commented on HADOOP-2144: - We should focus this test. Let's get say 32 distinct 128 MB files. Let's make sure they are approximately evenly distributed. Then let's cat them all. This should saturate the disks and/or CPU quite effectively. Yes? Let's post comparable numbers for straight cat from files and do some back of the envelope for what ratio we think a well implemented system might achieve. ok? This is very interesting. Data node process consumes 180% cpu Key: HADOOP-2144 URL: https://issues.apache.org/jira/browse/HADOOP-2144 Project: Hadoop Issue Type: Improvement Components: dfs Reporter: Runping Qi I did a test on DFS read throughput and found that the data node process consumes up to 180% cpu when it is under heavi load. Here are the details: The cluster has 380+ machines, each with 3GB mem and 4 cpus and 4 disks. I copied a 10GB file to dfs from one machine with a data node running there. Based on the dfs block placement policy, that machine has one replica for each block of the file. then I run 4 of the following commands in parellel: hadoop dfs -cat thefile /dev/null Since all the blocks have a local replica, all the read requests went to the local data node. I observed that: The data node process's cpu usage was around 180% for most of the time . The clients's cpu usage was moderate (as it should be). All the four disks were working concurrently with comparable read throughput. The total read throughput was maxed at 90MB/Sec, about 60% of the expected total aggregated max read throughput of 4 disks (160MB/Sec). Thus disks were not a bottleneck in this case. The data node's cpu usage seems unreasonably high. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2093) DFS should provide partition information for blocks, and map/reduce should schedule avoid schedule mappers with the splits off the same file system partition at the sam
[ https://issues.apache.org/jira/browse/HADOOP-2093?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12539753 ] eric baldeschwieler commented on HADOOP-2093: - An easier solution might simply be to schedule more blocks to be read at once. This will saturate the disk system with less complexity... DFS should provide partition information for blocks, and map/reduce should schedule avoid schedule mappers with the splits off the same file system partition at the same time -- Key: HADOOP-2093 URL: https://issues.apache.org/jira/browse/HADOOP-2093 Project: Hadoop Issue Type: New Feature Components: dfs, mapred Reporter: Runping Qi The summary is a bit of long. But the basic idea is to better utilize multiple file system partitions. For example, in a map reduce job, if we have 100 splits local to a node, and these 100 splits spread across 4 file system partitions, if we allow 4 mappers running concurrently, it is better that mappers each work on splits on different file system partitions. If in the worst case, all the mappers work on the splits on the same file system partition, then the other three file systems are not utilized at all. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1298) adding user info to file
[ https://issues.apache.org/jira/browse/HADOOP-1298?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12539794 ] eric baldeschwieler commented on HADOOP-1298: - Please forward me the bug, I filter out most jira, ok? adding user info to file Key: HADOOP-1298 URL: https://issues.apache.org/jira/browse/HADOOP-1298 Project: Hadoop Issue Type: New Feature Components: dfs, fs Reporter: Kurtis Heimerl Assignee: Christophe Taton Attachments: 1298_2007-09-22_1.patch, 1298_2007-10-04_1.patch, hadoop-user-munncha.patch17 I'm working on adding a permissions model to hadoop's DFS. The first step is this change, which associates user info with files. Following this I'll assoicate permissions info, then block methods based on that user info, then authorization of the user info. So, right now i've implemented adding user info to files. I'm looking for feedback before I clean this up and make it offical. I wasn't sure what release, i'm working off trunk. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1985) Abstract node to switch mapping into a topology service class used by namenode and jobtracker
[ https://issues.apache.org/jira/browse/HADOOP-1985?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12539020 ] eric baldeschwieler commented on HADOOP-1985: - I agree that an exec as a simple to config option should be required. Abstract node to switch mapping into a topology service class used by namenode and jobtracker - Key: HADOOP-1985 URL: https://issues.apache.org/jira/browse/HADOOP-1985 Project: Hadoop Issue Type: New Feature Reporter: eric baldeschwieler Assignee: Devaraj Das In order to implement switch locality in MapReduce, we need to have switch location in both the namenode and job tracker. Currently the namenode asks the data nodes for this info and they run a local script to answer this question. In our environment and others that I know of there is no reason to push this to each node. It is easier to maintain a centralized script that maps node DNS names to switch strings. I propose that we build a new class that caches known DNS name to switch mappings and invokes a loadable class or a configurable system call to resolve unknown DNS to switch mappings. We can then add this to the namenode to support the current block to switch mapping needs and simplify the data nodes. We can also add this same callout to the job tracker and then implement rack locality logic there without needing to chane the filesystem API or the split planning API. Not only is this the least intrusive path to building racklocal MR I can ID, it is also future compatible to future infrastructures that may derive topology on the fly, etc, etc... -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2012) Periodic verification at the Datanode
[ https://issues.apache.org/jira/browse/HADOOP-2012?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12538683 ] eric baldeschwieler commented on HADOOP-2012: - what value is there in scanning a single block continuously in the extreme case? Seems like we should do our part to keep our carbon footprint down. Periodic verification at the Datanode - Key: HADOOP-2012 URL: https://issues.apache.org/jira/browse/HADOOP-2012 Project: Hadoop Issue Type: New Feature Components: dfs Reporter: Raghu Angadi Assignee: Raghu Angadi Fix For: 0.16.0 Attachments: HADOOP-2012.patch, HADOOP-2012.patch, HADOOP-2012.patch, HADOOP-2012.patch Currently on-disk data corruption on data blocks is detected only when it is read by the client or by another datanode. These errors are detected much earlier if datanode can periodically verify the data checksums for the local blocks. Some of the issues to consider : - How should we check the blocks ( no more often than once every couple of weeks ?) - How do we keep track of when a block was last verfied ( there is a .meta file associcated with each lock ). - What action to take once a corruption is detected - Scanning should be done as a very low priority with rest of the datanode disk traffic in mind. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1985) Abstract node to switch mapping into a topology service class used by namenode and jobtracker
[ https://issues.apache.org/jira/browse/HADOOP-1985?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12538857 ] eric baldeschwieler commented on HADOOP-1985: - works for me Abstract node to switch mapping into a topology service class used by namenode and jobtracker - Key: HADOOP-1985 URL: https://issues.apache.org/jira/browse/HADOOP-1985 Project: Hadoop Issue Type: New Feature Reporter: eric baldeschwieler Assignee: Devaraj Das In order to implement switch locality in MapReduce, we need to have switch location in both the namenode and job tracker. Currently the namenode asks the data nodes for this info and they run a local script to answer this question. In our environment and others that I know of there is no reason to push this to each node. It is easier to maintain a centralized script that maps node DNS names to switch strings. I propose that we build a new class that caches known DNS name to switch mappings and invokes a loadable class or a configurable system call to resolve unknown DNS to switch mappings. We can then add this to the namenode to support the current block to switch mapping needs and simplify the data nodes. We can also add this same callout to the job tracker and then implement rack locality logic there without needing to chane the filesystem API or the split planning API. Not only is this the least intrusive path to building racklocal MR I can ID, it is also future compatible to future infrastructures that may derive topology on the fly, etc, etc... -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2012) Periodic verification at the Datanode
[ https://issues.apache.org/jira/browse/HADOOP-2012?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12537517 ] eric baldeschwieler commented on HADOOP-2012: - A couple of comments: 1) the idea of not keeping scan times strikes me as really bad. Also randomizing the scan order weirds me out. I do think Raghu's point about modeling the meta files may be valid. Why not simply always scan blocks in numeric order starting from zero and log actions. On restart we can tail this log to find the last block validated and start from there. Inserted blocks are recent by definition, so it is ok if we don't get around to them til the next scan. 2) It seems to me it might be better to try to repair the block if possible, rather then just delete it. This avoids bad corner cases. It adds complexity though. Thoughts? A simple variant is just to copy a new version locally. 3) Throttling might simply entail spacing and scheduling when you scan the next block to complete within roughly two weeks. This would imply that we want to persist when the current scan started. If we do that, the penalty of scanning quickly might be fairly ignorable, when you consider the other variations in work load a DN is exposed to anyway. You'd want some rule like always wait 10x the time it took you to validate a block between blocks to avoid wierd corner cases where the node gets a huge number of blocks added near the end of the time period. Periodic verification at the Datanode - Key: HADOOP-2012 URL: https://issues.apache.org/jira/browse/HADOOP-2012 Project: Hadoop Issue Type: New Feature Components: dfs Reporter: Raghu Angadi Assignee: Raghu Angadi Fix For: 0.16.0 Attachments: HADOOP-2012.patch, HADOOP-2012.patch, HADOOP-2012.patch, HADOOP-2012.patch Currently on-disk data corruption on data blocks is detected only when it is read by the client or by another datanode. These errors are detected much earlier if datanode can periodically verify the data checksums for the local blocks. Some of the issues to consider : - How should we check the blocks ( no more often than once every couple of weeks ?) - How do we keep track of when a block was last verfied ( there is a .meta file associcated with each lock ). - What action to take once a corruption is detected - Scanning should be done as a very low priority with rest of the datanode disk traffic in mind. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2066) filenames with ':' colon throws java.lang.IllegalArgumentException
[ https://issues.apache.org/jira/browse/HADOOP-2066?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12536917 ] eric baldeschwieler commented on HADOOP-2066: - I feel like we are going down a rat hole here. How do we get our input syntax to not surprise users? users expect to be able to specify posix like file strings. If URL syntax is going to keep throwing surprises and cross platform issues at us, maybe we need to create a syntax that doesn;t pull in a lot of unwanted baggage? filenames with ':' colon throws java.lang.IllegalArgumentException -- Key: HADOOP-2066 URL: https://issues.apache.org/jira/browse/HADOOP-2066 Project: Hadoop Issue Type: Bug Components: dfs Reporter: lohit vijayarenu Attachments: 2066_20071022.patch, HADOOP-2066.patch File names containing colon : throws java.lang.IllegalArgumentException while LINUX file system supports it. $ hadoop dfs -put ./testfile-2007-09-24-03:00:00.gz filenametest Exception in thread main java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: testfile-2007-09-24-03:00:00.gz at org.apache.hadoop.fs.Path.initialize(Path.java:140) at org.apache.hadoop.fs.Path.init(Path.java:126) at org.apache.hadoop.fs.Path.init(Path.java:50) at org.apache.hadoop.fs.FileUtil.checkDest(FileUtil.java:273) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:117) at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:776) at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:757) at org.apache.hadoop.fs.FsShell.copyFromLocal(FsShell.java:116) at org.apache.hadoop.fs.FsShell.run(FsShell.java:1229) at org.apache.hadoop.util.ToolBase.doMain(ToolBase.java:187) at org.apache.hadoop.fs.FsShell.main(FsShell.java:1342) Caused by: java.net.URISyntaxException: Relative path in absolute URI: testfile-2007-09-24-03:00:00.gz at java.net.URI.checkPath(URI.java:1787) at java.net.URI.init(URI.java:735) at org.apache.hadoop.fs.Path.initialize(Path.java:137) ... 10 more Path(String pathString) when given a filename which contains ':' treats it as URI and selects anything before ':' as scheme, which in this case is clearly not a valid scheme. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2066) filenames with ':' colon throws java.lang.IllegalArgumentException
[ https://issues.apache.org/jira/browse/HADOOP-2066?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12535497 ] eric baldeschwieler commented on HADOOP-2066: - The API used does not look like a URI API, hence a URI quoting requirement also seems wrong. filenames with ':' colon throws java.lang.IllegalArgumentException -- Key: HADOOP-2066 URL: https://issues.apache.org/jira/browse/HADOOP-2066 Project: Hadoop Issue Type: Bug Components: dfs Reporter: lohit vijayarenu File names containing colon : throws java.lang.IllegalArgumentException while LINUX file system supports it. [EMAIL PROTECTED] ~]$ hadoop dfs -put ./testfile-2007-09-24-03:00:00.gz filenametest Exception in thread main java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: testfile-2007-09-24-03:00:00.gz at org.apache.hadoop.fs.Path.initialize(Path.java:140) at org.apache.hadoop.fs.Path.init(Path.java:126) at org.apache.hadoop.fs.Path.init(Path.java:50) at org.apache.hadoop.fs.FileUtil.checkDest(FileUtil.java:273) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:117) at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:776) at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:757) at org.apache.hadoop.fs.FsShell.copyFromLocal(FsShell.java:116) at org.apache.hadoop.fs.FsShell.run(FsShell.java:1229) at org.apache.hadoop.util.ToolBase.doMain(ToolBase.java:187) at org.apache.hadoop.fs.FsShell.main(FsShell.java:1342) Caused by: java.net.URISyntaxException: Relative path in absolute URI: testfile-2007-09-24-03:00:00.gz at java.net.URI.checkPath(URI.java:1787) at java.net.URI.init(URI.java:735) at org.apache.hadoop.fs.Path.initialize(Path.java:137) ... 10 more [EMAIL PROTECTED] ~]$ Path(String pathString) when given a filename which contains ':' treats it as URI and selects anything before ':' as scheme, which in this case is clearly not a valid scheme. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-2055) JobConf should have a setInputPathFilter(PathFilter filter) method
[ https://issues.apache.org/jira/browse/HADOOP-2055?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12535507 ] eric baldeschwieler commented on HADOOP-2055: - we support globing in input paths now. Doesn't that address this need? IE *.foo JobConf should have a setInputPathFilter(PathFilter filter) method -- Key: HADOOP-2055 URL: https://issues.apache.org/jira/browse/HADOOP-2055 Project: Hadoop Issue Type: New Feature Environment: all Reporter: Alejandro Abdelnur Priority: Minor It should be possible to set a PathFilter for the input to avoid taking certain files as input data within the input directories. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1881) Update documentation for hadoop's configuration post HADOOP-785
[ https://issues.apache.org/jira/browse/HADOOP-1881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12532948 ] eric baldeschwieler commented on HADOOP-1881: - Either would be fine, but I've certainly found a lot of utility in historic docs from other projects and this would provide maximum clarity. So my preference would be to post both java docs and user docs from head of trunk and also the last N releases. Update documentation for hadoop's configuration post HADOOP-785 --- Key: HADOOP-1881 URL: https://issues.apache.org/jira/browse/HADOOP-1881 Project: Hadoop Issue Type: Improvement Components: documentation Affects Versions: 0.15.0 Reporter: Arun C Murthy Assignee: Arun C Murthy Fix For: 0.15.0 Attachments: HADOOP-1881-2.patch, HADOOP-1881_1_20071004.patch, HADOOP-1881_3_20071005.patch With significant changes to hadoop's configuration since HADOOP-785 the documentation for it needs to be completely overhauled: a) Exhaustive and accurate javadocs, including some specific examples. b) Update the wiki: http://wiki.apache.org/lucene-hadoop/HowToConfigure to c) Importantly: Put up a page describing hadoop's configuration on the hadoop website (via forrest). Any thing else folks can think of? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1968) Wildcard input syntax (glob) should support {}
[ https://issues.apache.org/jira/browse/HADOOP-1968?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12532949 ] eric baldeschwieler commented on HADOOP-1968: - Why is glob going through path? It seems to me that a glob string is not a path string and shouldn't be processed as such. The resulting match is a list of paths. Wildcard input syntax (glob) should support {} -- Key: HADOOP-1968 URL: https://issues.apache.org/jira/browse/HADOOP-1968 Project: Hadoop Issue Type: Improvement Components: fs Affects Versions: 0.14.1 Reporter: eric baldeschwieler Assignee: Hairong Kuang Fix For: 0.15.0 Attachments: curlyGlob.patch, curlyGlob1.patch We have users who have organized data by day and would like to select several days in a single input specification. For example they would like to be able to say: '/data/2007{0830,0831,0901}/typeX/' To input 3 days data into map-reduce (or Pig in this case). (Also the use of regexp to resolve glob paterns looks like it might introduce some other bugs. I'd appreciate it if someone took another look at the code to see if there are any file characters that could be interpreted as regexp instructions). -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1881) Update documentation for hadoop's configuration post HADOOP-785
[ https://issues.apache.org/jira/browse/HADOOP-1881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12532700 ] eric baldeschwieler commented on HADOOP-1881: - In forest it will not need release specific sections since it will be version controlled. Change bars would be cool if supported, but that is icing. Update documentation for hadoop's configuration post HADOOP-785 --- Key: HADOOP-1881 URL: https://issues.apache.org/jira/browse/HADOOP-1881 Project: Hadoop Issue Type: Improvement Components: documentation Affects Versions: 0.15.0 Reporter: Arun C Murthy Assignee: Arun C Murthy Fix For: 0.15.0 Attachments: HADOOP-1881-2.patch, HADOOP-1881_1_20071004.patch, HADOOP-1881_3_20071005.patch With significant changes to hadoop's configuration since HADOOP-785 the documentation for it needs to be completely overhauled: a) Exhaustive and accurate javadocs, including some specific examples. b) Update the wiki: http://wiki.apache.org/lucene-hadoop/HowToConfigure to c) Importantly: Put up a page describing hadoop's configuration on the hadoop website (via forrest). Any thing else folks can think of? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (HADOOP-1985) Abstract node to switch mapping into a topology service class used by namenode and jobtracker
Abstract node to switch mapping into a topology service class used by namenode and jobtracker - Key: HADOOP-1985 URL: https://issues.apache.org/jira/browse/HADOOP-1985 Project: Hadoop Issue Type: New Feature Reporter: eric baldeschwieler In order to implement switch locality in MapReduce, we need to have switch location in both the namenode and job tracker. Currently the namenode asks the data nodes for this info and they run a local script to answer this question. In our environment and others that I know of there is no reason to push this to each node. It is easier to maintain a centralized script that maps node DNS names to switch strings. I propose that we build a new class that caches known DNS name to switch mappings and invokes a loadable class or a configurable system call to resolve unknown DNS to switch mappings. We can then add this to the namenode to support the current block to switch mapping needs and simplify the data nodes. We can also add this same callout to the job tracker and then implement rack locality logic there without needing to chane the filesystem API or the split planning API. Not only is this the least intrusive path to building racklocal MR I can ID, it is also future compatible to future infrastructures that may derive topology on the fly, etc, etc... -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1968) Wildcard input syntax (glob) should support {}
[ https://issues.apache.org/jira/browse/HADOOP-1968?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12531995 ] eric baldeschwieler commented on HADOOP-1968: - It is not a blocker, but it would resolve some user issues we'd really like to fix. If we can get it into 15, it would make some people happy. But I would not hold the release for this feature. Wildcard input syntax (glob) should support {} -- Key: HADOOP-1968 URL: https://issues.apache.org/jira/browse/HADOOP-1968 Project: Hadoop Issue Type: Improvement Affects Versions: 0.14.1 Reporter: eric baldeschwieler Fix For: 0.15.0 We have users who have organized data by day and would like to select several days in a single input specification. For example they would like to be able to say: '/data/2007{0830,0831,0901}/typeX/' To input 3 days data into map-reduce (or Pig in this case). (Also the use of regexp to resolve glob paterns looks like it might introduce some other bugs. I'd appreciate it if someone took another look at the code to see if there are any file characters that could be interpreted as regexp instructions). -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (HADOOP-1968) Wildcard input syntax (glob) should support {}
Wildcard input syntax (glob) should support {} -- Key: HADOOP-1968 URL: https://issues.apache.org/jira/browse/HADOOP-1968 Project: Hadoop Issue Type: Improvement Reporter: eric baldeschwieler We have users who have organized data by day and would like to select several days in a single input specification. For example they would like to be able to say: '/data/2007{0830,0831,0901}/typeX/' To input 3 days data into map-reduce (or Pig in this case). (Also the use of regexp to resolve glob paterns looks like it might introduce some other bugs. I'd appreciate it if someone took another look at the code to see if there are any file characters that could be interpreted as regexp instructions). -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1894) Add fancy graphs for mapred task statuses
[ https://issues.apache.org/jira/browse/HADOOP-1894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12530500 ] eric baldeschwieler commented on HADOOP-1894: - I don't think they are so slow that we need to reject them. Let's make sure they work ok on the windows with firefox and that the IE7 experience is not really bad. Notification you need a plugin is fine, lots of clutter on the screen would be bad. This is a great starting point. I'm sure we're going to see a lot of evolution from here. There is no requirement it be perfect. Only that it not cause major problems. Add fancy graphs for mapred task statuses - Key: HADOOP-1894 URL: https://issues.apache.org/jira/browse/HADOOP-1894 Project: Hadoop Issue Type: New Feature Components: mapred Affects Versions: 0.15.0 Reporter: Enis Soztutar Assignee: Enis Soztutar Fix For: 0.15.0 Attachments: fancygraph_v1.0.patch, fancygraph_v1.1.patch, fancygraph_v1.2.patch, fancygraph_v1.3.patch, mapreduce.png, percentage.png I whould like to add graphics for mapred task statuses. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1700) Append to files in HDFS
[ https://issues.apache.org/jira/browse/HADOOP-1700?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12530603 ] eric baldeschwieler commented on HADOOP-1700: - I think we can get away without supporting truncate in the medium term. Less is more... That said, I don't have a strong opinion on generation numbers vs length. I'd like to be sure that we are able to detect the stale block issue. Maybe a unique file ID? Does this proposal try to finish with fewer datanodes than requested or does it request new ones? Append to files in HDFS --- Key: HADOOP-1700 URL: https://issues.apache.org/jira/browse/HADOOP-1700 Project: Hadoop Issue Type: New Feature Components: dfs Reporter: stack Request for being able to append to files in HDFS has been raised a couple of times on the list of late. For one example, see http://www.nabble.com/HDFS%2C-appending-writes-status-tf3848237.html#a10916193. Other mail describes folks' workarounds because this feature is lacking: e.g. http://www.nabble.com/Loading-data-into-HDFS-tf423.html#a12039480 (Later on this thread, Jim Kellerman re-raises the HBase need of this feature). HADOOP-337 'DFS files should be appendable' makes mention of file append but it was opened early in the life of HDFS when the focus was more on implementing the basics rather than adding new features. Interest fizzled. Because HADOOP-337 is also a bit of a grab-bag -- it includes truncation and being able to concurrently read/write -- rather than try and breathe new life into HADOOP-337, instead, here is a new issue focused on file append. Ultimately, being able to do as the google GFS paper describes -- having multiple concurrent clients making 'Atomic Record Append' to a single file would be sweet but at least for a first cut at this feature, IMO, a single client appending to a single HDFS file letting the application manage the access would be sufficent. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1883) Adding versioning to Record I/O
[ https://issues.apache.org/jira/browse/HADOOP-1883?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12529014 ] eric baldeschwieler commented on HADOOP-1883: - Yupp. So we should all pause and discuss before integrating recordio into hadoop RPC. There are solutions to your concern, but they all add complexity. I'm not sure it makes sense to evolve recordio in that direction. Adding versioning to Record I/O --- Key: HADOOP-1883 URL: https://issues.apache.org/jira/browse/HADOOP-1883 Project: Hadoop Issue Type: New Feature Reporter: Vivek Ratan There is a need to add versioning support to Record I/O. Users frequently update DDL files, usually by adding/removing fields, but do not want to change the name of the data structure. They would like older newer deserializers to read as much data as possible. For example, suppose Record I/O is used to serialize/deserialize log records, each of which contains a message and a timestamp. An initial data definition could be as follows: {code} class MyLogRecord { ustring msg; long timestamp; } {code} Record I/O creates a class, _MyLogRecord_, which represents a log record and can serialize/deserialize itself. Now, suppose newer log records additionally contain a severity level. A user would want to update the definition for a log record but use the same class name. The new definition would be: {code} class MyLogRecord { ustring msg; long timestamp; int severity; } {code} Users would want a new deserializer to read old log records (and perhaps use a default value for the severity field), and an old deserializer to read newer log records (and skip the severity field). This requires some concept of versioning in Record I/O, or rather, the additional ability to read/write type information of a record. The following is a proposal to do this. Every Record I/O Record will have type information which represents how the record is structured (what fields it has, what types, etc.). This type information, represented by the class _RecordTypeInfo_, is itself serializable/deserializable. Every Record supports a method _getRecordTypeInfo()_, which returns a _RecordTypeInfo_ object. Users are expected to serialize this type information (by calling _RecordTypeInfo.serialize()_) in an appropriate fashion (in a separate file, for example, or at the beginning of a file). Using the same DDL as above, here's how we could serialize log records: {code} FileOutputStream fOut = new FileOutputStream(data.log); CsvRecordOutput csvOut = new CsvRecordOutput(fOut); ... // get the type information for MyLogRecord RecordTypeInfo typeInfo = MyLogRecord.getRecordTypeInfo(); // ask it to write itself out typeInfo.serialize(csvOut); ... // now, serialize a bunch of records while (...) { MyLogRecord log = new MyLogRecord(); // fill up the MyLogRecord object ... // serialize log.serialize(csvOut); } {code} In this example, the type information of a Record is serialized fist, followed by contents of various records, all into the same file. Every Record also supports a method that allows a user to set a filter for deserializing. A method _setRTIFilter()_ takes a _RecordTypeInfo_ object as a parameter. This filter represents the type information of the data that is being deserialized. When deserializing, the Record uses this filter (if one is set) to figure out what to read. Continuing with our example, here's how we could deserialize records: {code} FileInputStream fIn = new FileInputStream(data.log); // we know the record was written in CSV format CsvRecordInput csvIn = new CsvRecordInput(fIn); ... // we know the type info is written in the beginning. read it. RecordTypeInfo typeInfoFilter = new RecordTypeInfo(); // deserialize it typeInfoFilter.deserialize(csvIn); // let MyLogRecord know what to expect MyLogRecord.setRTIFilter(typeInfoFilter); // deserialize each record while (there is data in file) { MyLogRecord log = new MyLogRecord(); log.read(csvIn); ... } {code} The filter is optional. If not provided, the deserializer expects data to be in the same format as it would serialize. (Note that a filter can also be provided for serializing, forcing the serializer to write information in the format of the filter, but there is no use case for this functionality yet). What goes in the type information for a record? The type information for each field in a Record is made up of: 1. a unique field ID, which is the field name. 2. a type ID, which denotes the type of the field (int, string, map, etc). The type information for a composite type contains type information for each of its fields. This approach is somewhat similar to the one taken by [Facebook's
[jira] Commented: (HADOOP-1894) Add fancy graphs for mapred task statuses
[ https://issues.apache.org/jira/browse/HADOOP-1894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12529026 ] eric baldeschwieler commented on HADOOP-1894: - You could stack the three phases and use shade or make each phase a second progress bar show what % of tasks have competed that phase. I kind of like the idea of stacking 3 colors with each growing from 0% length to 33%. That should work visually. Of course we are more interested in outliers than every task, so maybe one could just show net aggregate progress (a histogram of 10% complete increments showing the % of tasks in each). This would look like a stereo graphics equalizer with the color flowing from bars on the left to bars in the right as the task progresses. Then we could just break out the 5 slowest, 5 median and 5 fastest tasks and show them in more detail (task ID and % progress) Add fancy graphs for mapred task statuses - Key: HADOOP-1894 URL: https://issues.apache.org/jira/browse/HADOOP-1894 Project: Hadoop Issue Type: New Feature Components: mapred Affects Versions: 0.15.0 Reporter: Enis Soztutar Assignee: Enis Soztutar Fix For: 0.15.0 Attachments: fancygraph_v1.0.patch, fancygraph_v1.1.patch, mapreduce.png, percentage.png I whould like to add graphics for mapred task statuses. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1298) adding user info to file
[ https://issues.apache.org/jira/browse/HADOOP-1298?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12528438 ] eric baldeschwieler commented on HADOOP-1298: - Sounds interesting. I'm not sure we would want to embed it in the name node. This would exacerbate our namenode scaling challenges, wouldn't it? adding user info to file Key: HADOOP-1298 URL: https://issues.apache.org/jira/browse/HADOOP-1298 Project: Hadoop Issue Type: New Feature Components: dfs, fs Reporter: Kurtis Heimerl Assignee: Christophe Taton Fix For: 0.15.0 Attachments: 1298_2007-09-06b.patch, 1298_2007-09-07g.patch, hadoop-user-munncha.patch17 I'm working on adding a permissions model to hadoop's DFS. The first step is this change, which associates user info with files. Following this I'll assoicate permissions info, then block methods based on that user info, then authorization of the user info. So, right now i've implemented adding user info to files. I'm looking for feedback before I clean this up and make it offical. I wasn't sure what release, i'm working off trunk. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1881) Update documentation for hadoop's configuration post HADOOP-785
[ https://issues.apache.org/jira/browse/HADOOP-1881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12527916 ] eric baldeschwieler commented on HADOOP-1881: - Documentation like this does not belong in the wiki. It belongs in forest and javadoc, so it can be versioned and released with the product, included in patces, etc. Please make sure that this documentation is done in forest or javadoc. Update documentation for hadoop's configuration post HADOOP-785 --- Key: HADOOP-1881 URL: https://issues.apache.org/jira/browse/HADOOP-1881 Project: Hadoop Issue Type: Improvement Components: documentation Affects Versions: 0.15.0 Reporter: Arun C Murthy Assignee: Arun C Murthy Fix For: 0.15.0 With significant changes to hadoop's configuration since HADOOP-785 the documentation for it needs to be completely overhauled: a) Exhaustive and accurate javadoc. b) Update the wiki: http://wiki.apache.org/lucene-hadoop/HowToConfigure c) Put up some documentation on the website Any thing else folks can think of? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1867) use single parameter to specify a node's available ram
[ https://issues.apache.org/jira/browse/HADOOP-1867?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12527803 ] eric baldeschwieler commented on HADOOP-1867: - it would be great if we could automagically choose reasonable defaults. We'll still want a memory for M-R override since issues like what other code you run aon a node will dictate your preferred RAM allocated to M-R. use single parameter to specify a node's available ram -- Key: HADOOP-1867 URL: https://issues.apache.org/jira/browse/HADOOP-1867 Project: Hadoop Issue Type: Improvement Reporter: Doug Cutting To simplify configuration, we should use a single parameter to indicate a node's available RAM. Sites should not need to adjust more than this single parameter to configure nodes available memory. In task JVMs, some significant percentage of the memory should be reserved for application code, with the remainder divided among various system buffers. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1894) Add fancy graphs for mapred task statuses
[ https://issues.apache.org/jira/browse/HADOOP-1894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12527565 ] eric baldeschwieler commented on HADOOP-1894: - This looks very interesting! Add fancy graphs for mapred task statuses - Key: HADOOP-1894 URL: https://issues.apache.org/jira/browse/HADOOP-1894 Project: Hadoop Issue Type: New Feature Components: mapred Affects Versions: 0.15.0 Reporter: Enis Soztutar Assignee: Enis Soztutar Fix For: 0.15.0 Attachments: fancygraph_v1.0.patch, mapreduce.png, percentage.png I whould like to add graphics for mapred task statuses. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1298) adding user info to file
[ https://issues.apache.org/jira/browse/HADOOP-1298?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12526026 ] eric baldeschwieler commented on HADOOP-1298: - Do we have a plan for how a user's ID gets assigned to files his reduce job creates, or how his map job's read his files? adding user info to file Key: HADOOP-1298 URL: https://issues.apache.org/jira/browse/HADOOP-1298 Project: Hadoop Issue Type: New Feature Components: dfs, fs Reporter: Kurtis Heimerl Assignee: Christophe Taton Fix For: 0.15.0 Attachments: 1298_2007-09-06b.patch, 1298_2007-09-07g.patch, hadoop-user-munncha.patch17 I'm working on adding a permissions model to hadoop's DFS. The first step is this change, which associates user info with files. Following this I'll assoicate permissions info, then block methods based on that user info, then authorization of the user info. So, right now i've implemented adding user info to files. I'm looking for feedback before I clean this up and make it offical. I wasn't sure what release, i'm working off trunk. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1700) Append to files in HDFS
[ https://issues.apache.org/jira/browse/HADOOP-1700?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12525894 ] eric baldeschwieler commented on HADOOP-1700: - Yes, I'd thought of using length too. We have had requests to support truncate as well. I'm on the fence on that one. It is simple and more provably correct if we don't do truncates and just track block length. But we've a significant client group that has had an interest in truncates. We'll do some chewing on that one and see if we want to propose one project that does truncates and appends (less disruption to get both features) or appends only and presumably defer our ambition to do truncates for at least a year. The combination of very high block replication counts and network partitions still feels like it might produce some kind of nasty cases here. Append to files in HDFS --- Key: HADOOP-1700 URL: https://issues.apache.org/jira/browse/HADOOP-1700 Project: Hadoop Issue Type: New Feature Components: dfs Reporter: stack Request for being able to append to files in HDFS has been raised a couple of times on the list of late. For one example, see http://www.nabble.com/HDFS%2C-appending-writes-status-tf3848237.html#a10916193. Other mail describes folks' workarounds because this feature is lacking: e.g. http://www.nabble.com/Loading-data-into-HDFS-tf423.html#a12039480 (Later on this thread, Jim Kellerman re-raises the HBase need of this feature). HADOOP-337 'DFS files should be appendable' makes mention of file append but it was opened early in the life of HDFS when the focus was more on implementing the basics rather than adding new features. Interest fizzled. Because HADOOP-337 is also a bit of a grab-bag -- it includes truncation and being able to concurrently read/write -- rather than try and breathe new life into HADOOP-337, instead, here is a new issue focused on file append. Ultimately, being able to do as the google GFS paper describes -- having multiple concurrent clients making 'Atomic Record Append' to a single file would be sweet but at least for a first cut at this feature, IMO, a single client appending to a single HDFS file letting the application manage the access would be sufficent. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1861) Update the how to configure HDFS documentation to include new features
[ https://issues.apache.org/jira/browse/HADOOP-1861?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12525895 ] eric baldeschwieler commented on HADOOP-1861: - How can we structure this documentation so that it is checked into SVN, versioned and updates go in patch files? This will allow commiters to inforce its up-to-date-ness. Does it make sense to create a .txt or long articles in java doc format? Update the how to configure HDFS documentation to include new features Key: HADOOP-1861 URL: https://issues.apache.org/jira/browse/HADOOP-1861 Project: Hadoop Issue Type: Improvement Components: dfs, documentation Reporter: dhruba borthakur There has been lots of cases where HDFS administrators have enquired about setup and configuration of HDFS. A recent question asked about configuring Datanodes to understand the rack that it belongs to. There is a wiki page that is out-of-date: http://wiki.apache.org/lucene-hadoop/HowToConfigure Some new things that come to mind are: 1. rack location for Datanodes. 2. Trash configuration 3. SecondaryNamenode config parameters -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1298) adding user info to file
[ https://issues.apache.org/jira/browse/HADOOP-1298?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12525297 ] eric baldeschwieler commented on HADOOP-1298: - Sounds like you've addressed my concerns. Thanks! We'll need to get a handle on the performance impact of adding all of this AccessControlContext to every method. Maybe someone here can run our name node benchmarks on your patch. Raghu? adding user info to file Key: HADOOP-1298 URL: https://issues.apache.org/jira/browse/HADOOP-1298 Project: Hadoop Issue Type: New Feature Components: dfs, fs Reporter: Kurtis Heimerl Assignee: Christophe Taton Fix For: 0.15.0 Attachments: 1298_2007-08-30a.patch, 1298_2007-09-05g.patch, hadoop-user-munncha.patch17 I'm working on adding a permissions model to hadoop's DFS. The first step is this change, which associates user info with files. Following this I'll assoicate permissions info, then block methods based on that user info, then authorization of the user info. So, right now i've implemented adding user info to files. I'm looking for feedback before I clean this up and make it offical. I wasn't sure what release, i'm working off trunk. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1700) Append to files in HDFS
[ https://issues.apache.org/jira/browse/HADOOP-1700?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12525365 ] eric baldeschwieler commented on HADOOP-1700: - Why would the name node not keep the timestamp/version? Seems to me it will need it to disambiguate new block reports and such. We can afford the bytes IMO. Those statements about reasonably synchronized clocks worry me. Especially if clients or datanodes are producing them. If the name node is not the source of truth, I'm afraid we can introduce weird cases. I think the next logical step here is for us to outline a more detail proposal for review. I think this is going to take a couple of distinct steps. We can outline those and get to agreement on the basics. Jim's request that a file exist if its producer dies before a close sounds like it would be easy, but then we get to flush semantics... I take it you don't just want the last complete 128M block Jim? If you want the last log line then you need something similar to what I outlined. Append to files in HDFS --- Key: HADOOP-1700 URL: https://issues.apache.org/jira/browse/HADOOP-1700 Project: Hadoop Issue Type: New Feature Components: dfs Reporter: stack Request for being able to append to files in HDFS has been raised a couple of times on the list of late. For one example, see http://www.nabble.com/HDFS%2C-appending-writes-status-tf3848237.html#a10916193. Other mail describes folks' workarounds because this feature is lacking: e.g. http://www.nabble.com/Loading-data-into-HDFS-tf423.html#a12039480 (Later on this thread, Jim Kellerman re-raises the HBase need of this feature). HADOOP-337 'DFS files should be appendable' makes mention of file append but it was opened early in the life of HDFS when the focus was more on implementing the basics rather than adding new features. Interest fizzled. Because HADOOP-337 is also a bit of a grab-bag -- it includes truncation and being able to concurrently read/write -- rather than try and breathe new life into HADOOP-337, instead, here is a new issue focused on file append. Ultimately, being able to do as the google GFS paper describes -- having multiple concurrent clients making 'Atomic Record Append' to a single file would be sweet but at least for a first cut at this feature, IMO, a single client appending to a single HDFS file letting the application manage the access would be sufficent. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1704) Throttling for HDFS Trash purging
[ https://issues.apache.org/jira/browse/HADOOP-1704?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12525255 ] eric baldeschwieler commented on HADOOP-1704: - so is this a real issue, or can we close it? Sounds like a hypothetical concern that has never been observed. Throttling for HDFS Trash purging - Key: HADOOP-1704 URL: https://issues.apache.org/jira/browse/HADOOP-1704 Project: Hadoop Issue Type: Bug Components: dfs Reporter: dhruba borthakur When HDFS Trash is enabled, deletion of a file/directory results in it being moved to the Trash directory. The Trash directory is periodically purged by the Namenode. This means that all files/directories that users deleted in the last Trash period, gets really deleted when the Trash purging occurs. This might cause a burst of file/directory deletions. The Namenode tracks blocks that belonged to deleted files in a data structure named RecentInvalidateSets. There is a possibility that Trash purging may cause this data structure to bloat, causing undesireable behaviour of the Namenode. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1298) adding user info to file
[ https://issues.apache.org/jira/browse/HADOOP-1298?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12524943 ] eric baldeschwieler commented on HADOOP-1298: - Past experience shows that it is much harder to take something away from a user community than to never give it to them. It is impossible to predict all the programmatic interactions with the FS that may be implemented, so I'd suggest we do the easiest thing to support first, not the most general thing. What if we store permissions only on directories and only check permissions on the containing directory? This seems very easy to document and implement. It would minimize the permission code that will run in the current critical sections in the namenode (where performance is a major concern for us) and would give us a lot of flexibility to distribute the namenodes responsibilities in the future. It would also reduce the RAM used by permissions by not adding any state to the file objects... adding user info to file Key: HADOOP-1298 URL: https://issues.apache.org/jira/browse/HADOOP-1298 Project: Hadoop Issue Type: New Feature Components: dfs, fs Reporter: Kurtis Heimerl Assignee: Christophe Taton Fix For: 0.15.0 Attachments: 1298_2007-08-30a.patch, hadoop-user-munncha.patch17 I'm working on adding a permissions model to hadoop's DFS. The first step is this change, which associates user info with files. Following this I'll assoicate permissions info, then block methods based on that user info, then authorization of the user info. So, right now i've implemented adding user info to files. I'm looking for feedback before I clean this up and make it offical. I wasn't sure what release, i'm working off trunk. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1700) Append to files in HDFS
[ https://issues.apache.org/jira/browse/HADOOP-1700?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12524988 ] eric baldeschwieler commented on HADOOP-1700: - Just wanted to pitch in some context... Jim stated in the opening of this bug that a single client writing would be enough to address this issue. I agree. But what we should be clearer about is the ultimate desired semantics for readers. I'd define success as having a single client doing appends and flushes as desired (say per line in a log file) and having multiple clients tail -f the file and see updates at a reasonable rate, IE soon after each flush or every 64k bytes or so with less than a seconds latency. This would let us build systems that log directly into HDFS and have related systems respond based on those log streams. This is where I'd like to see us get with this issue. Clearly getting there involves getting a handle on all the stuff already discussed in this thread. We also need to think carefully about the pipelining and protocol issues involved in making this work. We might want to break the protocol change issues into another discussion, but I want to make sure we don't converge on solutions that will not work considering fine grained flushes. Append to files in HDFS --- Key: HADOOP-1700 URL: https://issues.apache.org/jira/browse/HADOOP-1700 Project: Hadoop Issue Type: New Feature Components: dfs Reporter: stack Request for being able to append to files in HDFS has been raised a couple of times on the list of late. For one example, see http://www.nabble.com/HDFS%2C-appending-writes-status-tf3848237.html#a10916193. Other mail describes folks' workarounds because this feature is lacking: e.g. http://www.nabble.com/Loading-data-into-HDFS-tf423.html#a12039480 (Later on this thread, Jim Kellerman re-raises the HBase need of this feature). HADOOP-337 'DFS files should be appendable' makes mention of file append but it was opened early in the life of HDFS when the focus was more on implementing the basics rather than adding new features. Interest fizzled. Because HADOOP-337 is also a bit of a grab-bag -- it includes truncation and being able to concurrently read/write -- rather than try and breathe new life into HADOOP-337, instead, here is a new issue focused on file append. Ultimately, being able to do as the google GFS paper describes -- having multiple concurrent clients making 'Atomic Record Append' to a single file would be sweet but at least for a first cut at this feature, IMO, a single client appending to a single HDFS file letting the application manage the access would be sufficent. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1440) JobClient should not sort input-splits
[ https://issues.apache.org/jira/browse/HADOOP-1440?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12503017 ] eric baldeschwieler commented on HADOOP-1440: - I tend to agree with runping. The framework should reserve the right to execute maps in any order it chooses to. Nailing down execution order will limit our ability to optimize later. Also, why put sorting into user code? It sounds like the need is to name the reduces, not control their order. So why not address that directly? Perhaps outputs can be numbered according to their original submission order in the case of reducer none? This need not pin down execution order. Sounds like perhaps we should deprecate map.input.file now that a more uniform mechanism exists to get this info? JobClient should not sort input-splits -- Key: HADOOP-1440 URL: https://issues.apache.org/jira/browse/HADOOP-1440 Project: Hadoop Issue Type: Improvement Components: mapred Affects Versions: 0.12.3 Environment: All Reporter: Milind Bhandarkar Assignee: Milind Bhandarkar Fix For: 0.14.0 Currently, the JobClient sorts the InputSplits returned by InputFormat in descending order, so that the map tasks corresponding to larger input-splits are scheduled first for execution than smaller ones. However, this causes problems in applications that produce data-sets partitioned similarly to the input partition with -reducer NONE. With -reducer NONE, map task i produces part-i. Howver, in the typical applications that use -reducer NONE it should produce a partition that has the same index as the input parrtition. (Of course, this requires that each partition should be fed in its entirety to a map, rather than splitting it into blocks, but that is a separate issue.) Thus, sorting input splits should be either controllable via a configuration variable, or the FileInputFormat should sort the splits and JobClient should honor the order of splits. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1296) Improve interface to FileSystem.getFileCacheHints
[ https://issues.apache.org/jira/browse/HADOOP-1296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12491870 ] eric baldeschwieler commented on HADOOP-1296: - So are we comfortable returning 100s of thousands of records in a single RPC from the name node? Would it be better to return a max of 10k record at a time or some such limit with a clear restart policy? Or is it ok for a client to open a socket and suck that much data in one session. Clearly more RPCs is more aggregate work, just wondering about starvation, locking, CPU spikes and all the usual suspects. Improve interface to FileSystem.getFileCacheHints - Key: HADOOP-1296 URL: https://issues.apache.org/jira/browse/HADOOP-1296 Project: Hadoop Issue Type: Improvement Components: fs Reporter: Owen O'Malley Assigned To: dhruba borthakur The FileSystem interface provides a very limited interface for finding the location of the data. The current method looks like: String[][] getFileCacheHints(Path file, long start, long len) throws IOException which returns a list of block info where the block info consists of a list host names. Because the hints don't include the information about where the block boundaries are, map/reduce is required to call the name node for each split. I'd propose that we fix the naming a bit and make it: public class BlockInfo extends Writable { public long getStart(); public String[] getHosts(); } BlockInfo[] getFileHints(Path file, long start, long len) throws IOException; So that map/reduce can query about the entire file and get the locations in a single call. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1170) Very high CPU usage on data nodes because of FSDataset.checkDataDir() on every connect
[ https://issues.apache.org/jira/browse/HADOOP-1170?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12490156 ] eric baldeschwieler commented on HADOOP-1170: - The thing to understand is that we can not upgrade our cluster to HEAD with this patch committed. This patch breaks us. We'll try to move forward in the new issue rather than advocating rolling this back, but this patch did not address the concerns we raised in this bug and so we have a problem. I hope we can avoid this in the future. I'm not advocating rolling back because I agree that these checks were not the appropriate solution to the disk problems they solved. In case the context isn't clear, we frequently see individual drives go read only on our machines. This check was inserted to allow this problem to be detected early and avoid failed jobs cause by write failures. Very high CPU usage on data nodes because of FSDataset.checkDataDir() on every connect -- Key: HADOOP-1170 URL: https://issues.apache.org/jira/browse/HADOOP-1170 Project: Hadoop Issue Type: Bug Components: dfs Affects Versions: 0.11.2 Reporter: Igor Bolotin Fix For: 0.13.0 Attachments: 1170-v2.patch, 1170.patch While investigating performance issues in our Hadoop DFS/MapReduce cluster I saw very high CPU usage by DataNode processes. Stack trace showed following on most of the data nodes: [EMAIL PROTECTED] daemon prio=1 tid=0x2aaacb5b7bd0 nid=0x5940 runnable [0x4166a000..0x4166ac00] at java.io.UnixFileSystem.checkAccess(Native Method) at java.io.File.canRead(File.java:660) at org.apache.hadoop.util.DiskChecker.checkDir(DiskChecker.java:34) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:164) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSDir.checkDirTree(FSDataset.java:168) at org.apache.hadoop.dfs.FSDataset$FSVolume.checkDirs(FSDataset.java:258) at org.apache.hadoop.dfs.FSDataset$FSVolumeSet.checkDirs(FSDataset.java:339) - locked 0x2aaab6fb8960 (a org.apache.hadoop.dfs.FSDataset$FSVolumeSet) at org.apache.hadoop.dfs.FSDataset.checkDataDir(FSDataset.java:544) at org.apache.hadoop.dfs.DataNode$DataXceiveServer.run(DataNode.java:535) at java.lang.Thread.run(Thread.java:595) I understand that it would take a while to check the entire data directory - as we have some 180,000 blocks/files in there. But what really bothers me that from the code I see that this check is executed for every client connection to the DataNode - which also means for every task executed in the cluster. Once I commented out the check and restarted datanodes - the performance went up and CPU usage went down to reasonable level. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1069) Rename Hadoop record I/O to Jute
[ https://issues.apache.org/jira/browse/HADOOP-1069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12478857 ] eric baldeschwieler commented on HADOOP-1069: - So let's get some proposals for good names? Our goal is to assign Jute a name that is short, memorable and works with our goals of defining a top-level package that can be used in many contexts. Doesn't seem optimal. Jute seems to do a better job. Open to other suggestions. Rename Hadoop record I/O to Jute Key: HADOOP-1069 URL: https://issues.apache.org/jira/browse/HADOOP-1069 Project: Hadoop Issue Type: Improvement Components: record Affects Versions: 0.12.0 Environment: All Reporter: Milind Bhandarkar Assigned To: Milind Bhandarkar Fix For: 0.13.0 jute was the original name of the hadoop record i/o component. IMHO, it is easier to pronounce, easier to remember and has already stuck among its users. This renaming should be done while there isn't a large codebase using jute, otherwise it will be very difficult later. rcc will be renamed jrc (jute record compiler). -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-1065) DFS client warnings / info's should be centrally logged by name node
[ https://issues.apache.org/jira/browse/HADOOP-1065?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12478870 ] eric baldeschwieler commented on HADOOP-1065: - Sounds like we should always log a reason with a deletion DFS client warnings / info's should be centrally logged by name node Key: HADOOP-1065 URL: https://issues.apache.org/jira/browse/HADOOP-1065 Project: Hadoop Issue Type: Improvement Reporter: eric baldeschwieler When a client encounters an error condition that it warns / infos the user about, we should log that centrally, so that we can mine this info later. For example CRC errors or block write retries. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (HADOOP-1065) DFS client warnings / info's should be centrally logged by name node
DFS client warnings / info's should be centrally logged by name node Key: HADOOP-1065 URL: https://issues.apache.org/jira/browse/HADOOP-1065 Project: Hadoop Issue Type: Improvement Reporter: eric baldeschwieler When a client encounters an error condition that it warns / infos the user about, we should log that centrally, so that we can mine this info later. For example CRC errors or block write retries. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-958) Building Hadoop results in a lot of warnings
[ https://issues.apache.org/jira/browse/HADOOP-958?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12476379 ] eric baldeschwieler commented on HADOOP-958: cool! Building Hadoop results in a lot of warnings Key: HADOOP-958 URL: https://issues.apache.org/jira/browse/HADOOP-958 Project: Hadoop Issue Type: Improvement Reporter: eric baldeschwieler Attachments: example-warnings.patch We are getting hundreds of warnings right now. Most of these are a result of our transition to 1.5 and deprecated uses of generics. We should still fix these, since producing lots of warnings: A) Leads to the perception that our code is of low quality B) Can mask warnings that come from real issues. --- I suggest we do two things 1) Submit a patch or set of patches to clean this up 2) Change our patch tester to validate that the number of warnings per build did not go up with this patch -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (HADOOP-958) Building Hadoop results in a lot of warnings
Building Hadoop results in a lot of warnings Key: HADOOP-958 URL: https://issues.apache.org/jira/browse/HADOOP-958 Project: Hadoop Issue Type: Improvement Reporter: eric baldeschwieler We are getting hundreds of warnings right now. Most of these are a result of our transition to 1.5 and deprecated uses of generics. We should still fix these, since producing lots of warnings: A) Leads to the perception that our code is of low quality B) Can mask warnings that come from real issues. --- I suggest we do two things 1) Submit a patch or set of patches to clean this up 2) Change our patch tester to validate that the number of warnings per build did not go up with this patch -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (HADOOP-619) Unify Map-Reduce and Streaming to take the same globbed input specification
[ http://issues.apache.org/jira/browse/HADOOP-619?page=comments#action_12455541 ] eric baldeschwieler commented on HADOOP-619: owen: yes arkady: -input ...somepath/*/*.gz or somesuch should work in this proposal, so that should solve your problem, right? Maybe I should relax on allowing directories in glob patterns and directories. It is just that this sort of matching can be really messy on typos. Standard unix tools don't allow it. But I guess as long as it is not recursive, the limits of input formats and such should keep this from going too nuts. So I'm ok with owen's proposal. (No recursion, right?) Unify Map-Reduce and Streaming to take the same globbed input specification --- Key: HADOOP-619 URL: http://issues.apache.org/jira/browse/HADOOP-619 Project: Hadoop Issue Type: Improvement Components: mapred Reporter: eric baldeschwieler Assigned To: Sanjay Dahiya Right now streaming input is specified very differently from other map-reduce input. It would be good if these two apps could take much more similar input specs. In particular -input in streaming expects a file or glob pattern while MR takes a directory. It would be cool if both could take a glob patern of files and if both took a directory by default (with some patern excluded to allow logs, metadata and other framework output to be safely stored). We want to be sure that MR input is backward compatible over this change. I propose that a single file should be accepted as an input or a single directory. Globs should only match directories if the paterns is '/' terminated, to avoid massive inputs specified by mistake. Thoughts? -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-227) Namespace check pointing is not performed until the namenode restarts.
[ http://issues.apache.org/jira/browse/HADOOP-227?page=comments#action_12455530 ] eric baldeschwieler commented on HADOOP-227: Sounds a lot like a BTree and comes with all of the issues. Lots of IO and complexity. Reimplementing that seems like a bad idea. perhaps you can find a good java BTree, but this seems like a big, heavy piece of code. Why do we need to do this? Namespace check pointing is not performed until the namenode restarts. -- Key: HADOOP-227 URL: http://issues.apache.org/jira/browse/HADOOP-227 Project: Hadoop Issue Type: Bug Components: dfs Affects Versions: 0.2.0 Reporter: Konstantin Shvachko Assigned To: Milind Bhandarkar In current implementation when the name node starts, it reads its image file, then the edits file, and then saves the updated image back into the image file. The image file is never updated after that. In order to provide the system reliability reliability the namespace information should be check pointed periodically, and the edits file should be kept relatively small. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-619) Unify Map-Reduce and Streaming to take the same globbed input specification
[ http://issues.apache.org/jira/browse/HADOOP-619?page=comments#action_12455532 ] eric baldeschwieler commented on HADOOP-619: Perhaps we should just limit to either globing or a single directory per argument and simply drop directories from globbing? This seems fairly simple and not too restrictive in practice. I agree that if a directory is used we should exclude files starting with _. This will allow us to put metadata in output directories. I think we should also simply exclude subdirectories in input directories. Again, I doubt this will prove restrictive in practice. It seems to me we should error out if any glob matches no files or a listed input directory is not present. Perhaps we could provide another switch for an optional input in case users actual want a job to run if an input spec doesn't match any input. Unify Map-Reduce and Streaming to take the same globbed input specification --- Key: HADOOP-619 URL: http://issues.apache.org/jira/browse/HADOOP-619 Project: Hadoop Issue Type: Improvement Components: mapred Reporter: eric baldeschwieler Assigned To: Sanjay Dahiya Right now streaming input is specified very differently from other map-reduce input. It would be good if these two apps could take much more similar input specs. In particular -input in streaming expects a file or glob pattern while MR takes a directory. It would be cool if both could take a glob patern of files and if both took a directory by default (with some patern excluded to allow logs, metadata and other framework output to be safely stored). We want to be sure that MR input is backward compatible over this change. I propose that a single file should be accepted as an input or a single directory. Globs should only match directories if the paterns is '/' terminated, to avoid massive inputs specified by mistake. Thoughts? -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-759) TextInputFormat should allow different treatment on carriage return char '\r'
[ http://issues.apache.org/jira/browse/HADOOP-759?page=comments#action_12454544 ] eric baldeschwieler commented on HADOOP-759: Is there a situation where we want to treat CR-LF (\r\n right?) as two line breaks? If we can afford the extra processing, perhaps we should just check for this case when we see a CR in get line? In the average case of only \n this will not cost us anything and we'll get CR-LF right for PC files. I don't think there is a case we will get wrong and we'll only incur extra processing for CR only files, which are rather rare I expect, since apple abandoned this with osX and I'm not aware of any current system that uses this convention... Just getting this right seems simpler than adding extra methods and complexity to the interface. Thoughts? TextInputFormat should allow different treatment on carriage return char '\r' - Key: HADOOP-759 URL: http://issues.apache.org/jira/browse/HADOOP-759 Project: Hadoop Issue Type: Improvement Reporter: Runping Qi The current implementation treat '\r' and '\n' both as line breakers. However, in some cases, it is desiable to strictly use '\n' as the solely line breaker and treat '\r' as a part of data in a line. One way to do this is to make readline function as a member function so that the user can create a subclass to overwrite the function with the desired behavior. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-761) Unit tests should cleanup created files in /tmp. It causes tests to fail if more than one users run tests on same machine.
[ http://issues.apache.org/jira/browse/HADOOP-761?page=comments#action_12454548 ] eric baldeschwieler commented on HADOOP-761: Using /tmp is simply not a good idea. It leaves you vulnerable to various of system configuration differences, of which there are millions out there. We should create temp files in the test's working directory. Unit tests should cleanup created files in /tmp. It causes tests to fail if more than one users run tests on same machine. -- Key: HADOOP-761 URL: http://issues.apache.org/jira/browse/HADOOP-761 Project: Hadoop Issue Type: Bug Components: test Reporter: Sanjay Dahiya Priority: Minor TestMiniMRLocalFS test cases creates /tmp/wc/input/, which is not cleaned up, any other user running test on same machine simultaneously or later will see test failures. It should either use a temp directory in the build/test or use java's File.createTempFile method for temporary data and then clean it up. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-761) Unit tests should cleanup created files in /tmp. It causes tests to fail if more than one users run tests on same machine.
[ http://issues.apache.org/jira/browse/HADOOP-761?page=comments#action_12454549 ] eric baldeschwieler commented on HADOOP-761: We should start a unit test authors document and add this to it! Unit tests should cleanup created files in /tmp. It causes tests to fail if more than one users run tests on same machine. -- Key: HADOOP-761 URL: http://issues.apache.org/jira/browse/HADOOP-761 Project: Hadoop Issue Type: Bug Components: test Reporter: Sanjay Dahiya Priority: Minor TestMiniMRLocalFS test cases creates /tmp/wc/input/, which is not cleaned up, any other user running test on same machine simultaneously or later will see test failures. It should either use a temp directory in the build/test or use java's File.createTempFile method for temporary data and then clean it up. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-738) dfs get or copyToLocal should not copy crc file
[ http://issues.apache.org/jira/browse/HADOOP-738?page=comments#action_12452613 ] eric baldeschwieler commented on HADOOP-738: Well the discussion does seem to be on the topic of the subject. As discussed in the thread there are several reasons to consider the change. Enhancing the description could be done. I agree that we probably should fix the interaction of the CRC files in the current copy too. Although renaming them so they are visible would help a lot no mater what. I also agree that we will need sub-block CRC info even when we move the CRC data to be a block attribute. Moving multi-terrabyte objects to local disk is not the prototypical use of Hadoop in our environment. It is certainly not the prototypical reason we invoke -get or -copyToLocal. We don't seem to be converging here. Maybe we should create two commands. One which is typically used for lightweight copies and does not write CRCs and one which does and has an inverse command that validates the CRCs on import. Although what happens when a CRC does not match on import? The CRC exporter would create visible CRCs and would have well defined semantics for overwriting files. (Failing if the target directory would avoid the problem in the description ...) dfs get or copyToLocal should not copy crc file --- Key: HADOOP-738 URL: http://issues.apache.org/jira/browse/HADOOP-738 Project: Hadoop Issue Type: Bug Components: dfs Affects Versions: 0.8.0 Environment: all Reporter: Milind Bhandarkar Assigned To: Milind Bhandarkar Fix For: 0.9.0 Attachments: hadoop-crc.patch Currently, when we -get or -copyToLocal a directory from DFS, all the files including crc files are also copied. When we -put or -copyFromLocal again, since the crc files already exist on DFS, this put fails. The solution is not to copy checksum files when copying to local. Patch is forthcoming. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-746) CRC computation and reading should move into a nested FileSystem
[ http://issues.apache.org/jira/browse/HADOOP-746?page=comments#action_12452615 ] eric baldeschwieler commented on HADOOP-746: interesting. This seems like it would have advantages in how we manage temporary storage and such. BUT... I think HDFS needs to support CRCs for all files below the covers. I don't think we should rip that out and leave it to use code to invoke CRCs. The FS needs CRCs to manage replication and validation and should have a uniform internal mechanism. I don't know that it is necessary that these CRCs be user accessible, but I do think that it is necessary that all blocks CRC in the same simple way. Nor is that URL very friendly... Also nominally the URI prefix is supposted to specify the protocol / transport, right? CRC seems like it belongs below the transport, not wrapping it. odd. CRC computation and reading should move into a nested FileSystem Key: HADOOP-746 URL: http://issues.apache.org/jira/browse/HADOOP-746 Project: Hadoop Issue Type: Improvement Components: fs Affects Versions: 0.8.0 Reporter: Owen O'Malley Assigned To: Owen O'Malley Currently FileSystem provides both an interface and a mechanism for computing and checking crc files. I propose splitting the crc code into a nestable FileSystem that like the PhasedFileSystem has a backing FileSystem. Once the Paths are converted to URI, this is fairly natural to express. To use crc files, your uris will look like: crc://hdfs:%2f%2fhost1:8020/ which is a crc FileSystem with an underlying file system of hdfs://host1:8020 This will allow users to use crc files where they make sense for their application/cluster and get rid of the raw methods. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-738) dfs get or copyToLocal should not copy crc file
[ http://issues.apache.org/jira/browse/HADOOP-738?page=comments#action_12451824 ] eric baldeschwieler commented on HADOOP-738: The issues you raise are real when you are moving multi-terabyte search indexes and such. Having end-2-end CRC support is crucial and I'm all for keeping it in. But yes, our average use is not sophisticated in that their average operation to local disk in our environment is just to pull some experiment results locally. In general modern machines are good enough that megabyte to gigabyte sized files can be read and written reliably enough that CRC errors are not a dominate concern. Where they are a concern, I suggest we make the CRC files visible. Rather than using dot files, just append .crc to the file name. Then at least folks can see them and ask the right questions, etc. Things will work better that way. dfs get or copyToLocal should not copy crc file --- Key: HADOOP-738 URL: http://issues.apache.org/jira/browse/HADOOP-738 Project: Hadoop Issue Type: Bug Components: dfs Affects Versions: 0.8.0 Environment: all Reporter: Milind Bhandarkar Assigned To: Milind Bhandarkar Fix For: 0.9.0 Attachments: hadoop-crc.patch Currently, when we -get or -copyToLocal a directory from DFS, all the files including crc files are also copied. When we -put or -copyFromLocal again, since the crc files already exist on DFS, this put fails. The solution is not to copy checksum files when copying to local. Patch is forthcoming. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-442) slaves file should include an 'exclude' section, to prevent bad datanodes and tasktrackers from disrupting a cluster
[ http://issues.apache.org/jira/browse/HADOOP-442?page=comments#action_12450506 ] eric baldeschwieler commented on HADOOP-442: Current proposal: - Add config variables that points to file containing list of nodes HDFS should expect (slaves file) (optional config) - Add config variable that points to a file containing a list of excluded nodes (from previous list) (optional config) - The nameNode reads these files on startup (iff config). It keeps a list of included nodes and another of excluded nodes. If the include list is configured, it will be tested when a node registers or heartbeats. If the node is not on the list, it will be told to shutdown on the response. If the exclude list is configured, than a node will also be shutdown if listed. - We will add an admin command to re-read the inclusion and exclusion files - The job tracker will also read these lists and have a new admin command to reread the files slaves file should include an 'exclude' section, to prevent bad datanodes and tasktrackers from disrupting a cluster --- Key: HADOOP-442 URL: http://issues.apache.org/jira/browse/HADOOP-442 Project: Hadoop Issue Type: Bug Components: conf Reporter: Yoram Arnon I recently had a few nodes go bad, such that they were inaccessible to ssh, but were still running their java processes. tasks that executed on them were failing, causing jobs to fail. I couldn't stop the java processes, because of the ssh issue, so I was helpless until I could actually power down these nodes. restarting the cluster doesn't help, even when removing the bad nodes from the slaves file - they just reconnect and are accepted. while we plan to avoid tasks from launching on the same nodes over and over, what I'd like is to be able to prevent rogue processes from connecting to the masters. Ideally, the slaves file will contain an 'exclude' section, which will list nodes that shouldn't be accessed, and should be ignored if they try to connect. That would also help in configuring the slaves file for a large cluster - I'd list the full range of machines in the cluster, then list the ones that are down in the 'exclude' section -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-723) Race condition exists in the method MapOutputLocation.getFile
[ http://issues.apache.org/jira/browse/HADOOP-723?page=comments#action_12450178 ] eric baldeschwieler commented on HADOOP-723: You might want to use a temp file and rename strategy to make sure: a) you only delete files you own b) it is clear that a file was completely transfered by inspection. c) temp files can be easily cleaned up in some conditions (restart say) Race condition exists in the method MapOutputLocation.getFile - Key: HADOOP-723 URL: http://issues.apache.org/jira/browse/HADOOP-723 Project: Hadoop Issue Type: Bug Components: mapred Reporter: Devaraj Das There seems to be a race condition in the way the Reduces copy the map output files from the Maps. If a copier is blocked in the connect method (in the beginning of the method MapOutputLocation.getFile) to a Jetty on a Map, and the MapCopyLeaseChecker detects that the copier was idle for too long, it will go ahead and issue a interrupt (read 'kill') to this thread and create a new Copier thread. However, the copier, currently blocked trying to connect to Jetty on a Map, doesn't actually get killed until the connect timeout expires and as soon as the connect comes out (with an IOException), it will delete the map output file which actually could have been (successfully) created by the new Copier thread. This leads to the Sort phase for that reducer failing with a FileNotFoundException. One simple way to fix this is to not delete the file if the file was not created within this getFile method. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-686) job.setOutputValueComparatorClass(theClass) should be supported
[ http://issues.apache.org/jira/browse/HADOOP-686?page=comments#action_12447621 ] eric baldeschwieler commented on HADOOP-686: There exist hacks to achieve this effect now. I'll try to scare up an expert to document them. Also a good long term feature. job.setOutputValueComparatorClass(theClass) should be supported --- Key: HADOOP-686 URL: http://issues.apache.org/jira/browse/HADOOP-686 Project: Hadoop Issue Type: New Feature Components: mapred Environment: all environment Reporter: Feng Jiang if the input of Reduce phase is : K2, V3 K2, V2 K1, V5 K1, V3 K1, V4 in the current hadoop, the reduce output could be: K1, (V5, V3, V4) K2, (V3, V2) But I hope hadoop supports job.setOutputValueComparatorClass(theClass), so that i can make values are in order, and the output could be: K1, (V3, V4, V5) K2, (V2, V3) This feature is very important, I think. Without it, we have to take the sorting by ourselves, and have to worry about the possibility that the values are too large to fit into memory. Then the codes becomes too hard to read. That is the reason why i think this feature is so important, and should be done in the hadoop framework. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-681) Adminstrative hook to pull live nodes out of a HDFS cluster
[ http://issues.apache.org/jira/browse/HADOOP-681?page=comments#action_12447372 ] eric baldeschwieler commented on HADOOP-681: Looks good overall. Some comments on the details... The tranquil state should be persistent IMO. This should be recorded to disk and backed up anywhere we backup changelog data. the dfsshell command is for user commands, admin commands should have a distinct API to avoid confusion allow later divergence in security / etc. I'd prefer it if remote tools could not be used without changing the state on the namenode to effect its operation. This is a simple way of proving that the reconfig is valid. For example, change a file listing the idled nodes and then ping the server to reparse it. HDFS is not stable without a stable definition of the cluster, we need to stop building APIs that ignore this. BTW tranquil is a little obscure. What about idle or decommissioned or toBeKilled? Adminstrative hook to pull live nodes out of a HDFS cluster --- Key: HADOOP-681 URL: http://issues.apache.org/jira/browse/HADOOP-681 Project: Hadoop Issue Type: New Feature Components: dfs Affects Versions: 0.8.0 Reporter: dhruba borthakur Assigned To: dhruba borthakur Introduction An administrator sometimes needs to bring down a datanode for scheduled maintenance. It would be nice if HDFS can be informed about this event. On receipt of this event, HDFS can take steps so that HDFS data is not lost when the node goes down at a later time. Architecture --- In the existing architecture, a datanode can be in one of two states: dead or alive. A datanode is alive if its heartbeats are being processed by the namenode. Otherwise that datanode is in dead state. We extend the architecture to introduce the concept of a tranquil state for a datanode. A datanode is in tranquil state if: - it cannot be a target for replicating any blocks - any block replica that it currently contains does not count towards the target-replication-factor of that block Thus, a node that is in tranquil state can be brought down without impacting the guarantees provided by HDFS. The tranquil state is not persisted across namenode restarts. If the namenode restarts then that datanode will go back to being in the dead or alive state. The datanode is completely transparent to the fact that it has been labeled as being in tranquil state. It can continue to heartbeat and serve read requests for datablocks. DFSShell Design --- We extend the DFS Shell utility to specify a list of nodes to the namenode. hadoop dfs -tranquil {set|clear|get} datanodename1 [,datanodename2] The DFSShell utility sends this list to the namenode. This DFSShell command invoked with the set option completes when the list is transferred to the namenode. This command is non-blocking; it returns before the datanode is actually in the tranquil state. The client can then query the state by re-issuing the command with the get option. This option will indicate whether the datanode is in tranquil state or is being tranquiled. The clear option is used to transition a tranquil datanode to the alive state. The clear option is a no-op if the datanode is not in the tranquil state. ClientProtocol Design The ClientProtocol is the protocol exported by the namenode for its client. This protocol is extended to incorporate three new methods: ClientProtocol.setTranquil(String[] datanodes) ClientProtocol.getTranquil(String datanode) ClientProtocol.clearTranquil(String[] datanodes) The ProtocolVersion is incremented to prevent conversations between imcompatible clients and servers. An old DFSShell cannot talk to the new NameNode and vice-versa. NameNode Design - The namenode does the bulk of the work for supporting this new feature. The DatanodeInfo object has a new private member named state. It also has three new member functions: datanodeInfo.tranquilStarted(): start the process of tranquilization datanodeInfo.tranquilCompleted(): node is not in tranquil state datanodeInfo.clearTranquil() : remove tranquilization from node The namenode exposes a new API to set and clear tranquil states for a datanode. On receipt of a set tranquil command, it invokes datanodeInfo.tranquilStarted(). The FSNamesystem.chooseTarget() method skips over datanodes that are marked as being in the tranquil state. This ensures that tranquil-datanodes are never chosen as targets of replication. The namenode does *not* record this operation in either the FsImage or the EditLogs. The namenode puts all the blocks from a being-tranquiled node into the neededReplication data structure.
[jira] Commented: (HADOOP-667) Path configuration properties should not be comma separated
[ http://issues.apache.org/jira/browse/HADOOP-667?page=comments#action_12446468 ] eric baldeschwieler commented on HADOOP-667: humm, so how do we specify a list of paths? sounds like a conversation about quoting conventions is needed Path configuration properties should not be comma separated --- Key: HADOOP-667 URL: http://issues.apache.org/jira/browse/HADOOP-667 Project: Hadoop Issue Type: Bug Components: fs Affects Versions: 0.7.2 Reporter: Nigel Daley Priority: Minor A few configuration properties allow multiple directory paths separated by comma's (,). Since comma is a valid character for a directory name, it should not be used as a path separator. At a minimum, this applies to these properties: mapred.local.dir dfs.name.dir [ I also wonder how robust the implementation is against paths that contain spaces. ] -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-654) jobs fail with some hardware/system failures on a small number of nodes
[ http://issues.apache.org/jira/browse/HADOOP-654?page=comments#action_12445697 ] eric baldeschwieler commented on HADOOP-654: # failures should be visible by node on the job tracker UI. Nodes that are not getting jobs should be highlighted on the UI. jobs fail with some hardware/system failures on a small number of nodes --- Key: HADOOP-654 URL: http://issues.apache.org/jira/browse/HADOOP-654 Project: Hadoop Issue Type: Bug Components: mapred Affects Versions: 0.7.2 Reporter: Yoram Arnon Assigned To: Owen O'Malley Priority: Minor occasionally, such as when the OS is out of some resource, a node fails only partly. The node is up and running, the task tracker is running and sending heartbeats, but every task fails because the tasktracker can't fork tasks or something. In these cases, that task tracker keeps getting assigned tasks to execute, and they all fail. A couple of nodes like that and jobs start failing badly. The job tracker should avoid assigning tasks to tasktrackers that are misbehaving. simple approach: avoid tasktrackers that report many more failures than average (say 3X). Simply use the info sent by the TT. better but harder: track TT failures over time and: 1. avoid those that exhibit a high failure *rate* 2. tell them to shut down -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-645) Map-reduce task does not finish correctly when -reducer NONE is specified
[ http://issues.apache.org/jira/browse/HADOOP-645?page=comments#action_12445271 ] eric baldeschwieler commented on HADOOP-645: Is this a streaming specific discussion, or does general map-reduce really imply that map outputs go directly to HDFS when no reducers are run? I'm not sure that is a viable scalable behavior. This may simply create way too many outputs and kill the FS. Do we believe this is the current behavior or an ask for an extension? Map-reduce task does not finish correctly when -reducer NONE is specified - Key: HADOOP-645 URL: http://issues.apache.org/jira/browse/HADOOP-645 Project: Hadoop Issue Type: Bug Components: contrib/streaming Affects Versions: 0.7.2 Reporter: dhruba borthakur Assigned To: dhruba borthakur Map-reduce task does not finish correctly when -reducer NONE is specified, The NONE option means that the reducer should not be generating any output. Using this option causes an exception in the task tracker: java.lang.IllegalArgumentException: URI is not hierarchical TaskRunner: at java.io.File.init(File.java:335) TaskRunner: at org.apache.hadoop.streaming.PipeMapRed.mapRedFinished(PipeMapRed.java:583) TaskRunner: at org.apache.hadoop.streaming.PipeMapper.close(PipeMapper.java:96) TaskRunner: at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:49) TaskRunner: at org.apache.hadoop.mapred.MapTask.run(MapTask.java:213) TaskRunner: at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1240) TaskRunner: sideEffectURI_ file:output length 11 -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Created: (HADOOP-619) Unify Map-Reduce and Streaming to take the same globbed input specification
Unify Map-Reduce and Streaming to take the same globbed input specification --- Key: HADOOP-619 URL: http://issues.apache.org/jira/browse/HADOOP-619 Project: Hadoop Issue Type: Improvement Reporter: eric baldeschwieler Right now streaming input is specified very differently from other map-reduce input. It would be good if these two apps could take much more similar input specs. In particular -input in streaming expects a file or glob pattern while MR takes a directory. It would be cool if both could take a glob patern of files and if both took a directory by default (with some patern excluded to allow logs, metadata and other framework output to be safely stored). We want to be sure that MR input is backward compatible over this change. I propose that a single file should be accepted as an input or a single directory. Globs should only match directories if the paterns is '/' terminated, to avoid massive inputs specified by mistake. Thoughts? -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-331) map outputs should be written to a single output file with an index
[ http://issues.apache.org/jira/browse/HADOOP-331?page=comments#action_12443086 ] eric baldeschwieler commented on HADOOP-331: Sounds good. I think we are converging on something. A couple of points: 1) When we spill, we don't need to keep any record of the spilled data. Devaraj maintained operating on these arrays after spills. We should not do that. Everything should be cleared out. (Except possibly the index of partitions to file offsets). 2) I like the idea of extending the block compressed sequence file to directly support flushes at partition boundaries and merges of ranges. This will be reused in reduce as doug observed. 3) We should not spill based on a number of records. Don't see any value in that. We should just spill based on RAM used. 4) Related, we need to track total RAM used, not just for values, but also for keys and arrays. We don't want the system to blow up in the degenerate cases of huge keys or null values and many, many keys. map outputs should be written to a single output file with an index --- Key: HADOOP-331 URL: http://issues.apache.org/jira/browse/HADOOP-331 Project: Hadoop Issue Type: Improvement Components: mapred Affects Versions: 0.3.2 Reporter: eric baldeschwieler Assigned To: Devaraj Das The current strategy of writing a file per target map is consuming a lot of unused buffer space (causing out of memory crashes) and puts a lot of burden on the FS (many opens, inodes used, etc). I propose that we write a single file containing all output and also write an index file IDing which byte range in the file goes to each reduce. This will remove the issue of buffer waste, address scaling issues with number of open files and generally set us up better for scaling. It will also have advantages with very small inputs, since the buffer cache will reduce the number of seeks needed and the data serving node can open a single file and just keep it open rather than needing to do directory and open ops on every request. The only issue I see is that in cases where the task output is substantiallyu larger than its input, we may need to spill multiple times. In this case, we can do a merge after all spills are complete (or during the final spill). -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-331) map outputs should be written to a single output file with an index
[ http://issues.apache.org/jira/browse/HADOOP-331?page=comments#action_12443092 ] eric baldeschwieler commented on HADOOP-331: A related feature. Compressing values individually as we serialize (before sort/spill) should be an option. Even if we are using block compression on output. Sounds inefficient, but if it reduces the number of spills 3x, this saves total disk passes and compression/decompression passes. Also clearly good if we choose to just compress by values, not block in the sequence files, since we could then just pass the compressed values about. map outputs should be written to a single output file with an index --- Key: HADOOP-331 URL: http://issues.apache.org/jira/browse/HADOOP-331 Project: Hadoop Issue Type: Improvement Components: mapred Affects Versions: 0.3.2 Reporter: eric baldeschwieler Assigned To: Devaraj Das The current strategy of writing a file per target map is consuming a lot of unused buffer space (causing out of memory crashes) and puts a lot of burden on the FS (many opens, inodes used, etc). I propose that we write a single file containing all output and also write an index file IDing which byte range in the file goes to each reduce. This will remove the issue of buffer waste, address scaling issues with number of open files and generally set us up better for scaling. It will also have advantages with very small inputs, since the buffer cache will reduce the number of seeks needed and the data serving node can open a single file and just keep it open rather than needing to do directory and open ops on every request. The only issue I see is that in cases where the task output is substantiallyu larger than its input, we may need to spill multiple times. In this case, we can do a merge after all spills are complete (or during the final spill). -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-603) Extend SequenceFile to provide MapFile function by storing index at the end of the file
[ http://issues.apache.org/jira/browse/HADOOP-603?page=comments#action_12442654 ] eric baldeschwieler commented on HADOOP-603: Yeah. Just keeping an array of offset vectors per partition would be simpler. Jim: Don't see a problem with your suggestion (HADOOP-603), but I don't think the two projects relate. These files will be broken into partitions. Fitting that into a MapFile might be a bit of a stretch. Extend SequenceFile to provide MapFile function by storing index at the end of the file --- Key: HADOOP-603 URL: http://issues.apache.org/jira/browse/HADOOP-603 Project: Hadoop Issue Type: Improvement Components: dfs Reporter: Jim Kellerman MapFile increases the load on the name node as two files are created to provide a index file format. If SequenceFile were extended by storing the index at the end of the file, 1/2 of the files currently created for a map/reduce operation would be needed, reducing the load on the name node. Perhaps this is why Google implemented SSTable files in this manner. (SSTable files are functionally identical to Hadoop MapFiles) (see the paper on BigTable - section 4 Building Blocks http://labs.google.com/papers/bigtable.html) -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Updated: (HADOOP-603) Extend SequenceFile to provide MapFile function by storing index at the end of the file
[ http://issues.apache.org/jira/browse/HADOOP-603?page=all ] eric baldeschwieler updated HADOOP-603: --- Comment: was deleted Extend SequenceFile to provide MapFile function by storing index at the end of the file --- Key: HADOOP-603 URL: http://issues.apache.org/jira/browse/HADOOP-603 Project: Hadoop Issue Type: Improvement Components: dfs Reporter: Jim Kellerman MapFile increases the load on the name node as two files are created to provide a index file format. If SequenceFile were extended by storing the index at the end of the file, 1/2 of the files currently created for a map/reduce operation would be needed, reducing the load on the name node. Perhaps this is why Google implemented SSTable files in this manner. (SSTable files are functionally identical to Hadoop MapFiles) (see the paper on BigTable - section 4 Building Blocks http://labs.google.com/papers/bigtable.html) -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-331) map outputs should be written to a single output file with an index
[ http://issues.apache.org/jira/browse/HADOOP-331?page=comments#action_12442655 ] eric baldeschwieler commented on HADOOP-331: Yeah. Just keeping an array of offset vectors per partition would be simpler. Jim: Don't see a problem with your suggestion (HADOOP-603), but I don't think the two projects relate. These files will be broken into partitions. Fitting that into a MapFile might be a bit of a stretch. map outputs should be written to a single output file with an index --- Key: HADOOP-331 URL: http://issues.apache.org/jira/browse/HADOOP-331 Project: Hadoop Issue Type: Improvement Components: mapred Affects Versions: 0.3.2 Reporter: eric baldeschwieler Assigned To: Devaraj Das The current strategy of writing a file per target map is consuming a lot of unused buffer space (causing out of memory crashes) and puts a lot of burden on the FS (many opens, inodes used, etc). I propose that we write a single file containing all output and also write an index file IDing which byte range in the file goes to each reduce. This will remove the issue of buffer waste, address scaling issues with number of open files and generally set us up better for scaling. It will also have advantages with very small inputs, since the buffer cache will reduce the number of seeks needed and the data serving node can open a single file and just keep it open rather than needing to do directory and open ops on every request. The only issue I see is that in cases where the task output is substantiallyu larger than its input, we may need to spill multiple times. In this case, we can do a merge after all spills are complete (or during the final spill). -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-331) map outputs should be written to a single output file with an index
[ http://issues.apache.org/jira/browse/HADOOP-331?page=comments#action_12442453 ] eric baldeschwieler commented on HADOOP-331: re: devaraj I like the approach. One refinement suggested below: I don't think you want to store the partkeys inline. That requires more code change and an on disk format changes and wasted bytes to disk and over the wire. I think you spill serialized key/values with a side file that maps each partition to a start offset. In RAM you spill serialized key/value pairs to your buffer and also keep an array/vector (apply appropriate java class here) of (partition,offset to key). You can then quicksort the array and spill. You want to be sure to be able to apply a block compressor to each partition as spilled. This will be very efficient and simple. So record the compressed lengths (kimoon suggested this on another thread). Merging would go as you outline. You could read one line of each sidefile and then merge the next partition from each, so the merge would only consider the keys. Since it would be per partition. You need the sidefile to support efficient access for the reduce readers anyway. --- re: brian's comments I think we should keep maps simple and focus this effort on reduces, which deal with much larger size. That said, a corner case with HUGE maps should have a reasonable outcome. I think we need a stripped file abstraction to deal with these cases, where outputs are placed in medium HDFS sized blocks on whichever disk makes the most sense. This same approach would probably be more used on the reduce side. But I think this should come as a second project, rather than burdening this work with it. Anyone want to file a bug on it? map outputs should be written to a single output file with an index --- Key: HADOOP-331 URL: http://issues.apache.org/jira/browse/HADOOP-331 Project: Hadoop Issue Type: Improvement Components: mapred Affects Versions: 0.3.2 Reporter: eric baldeschwieler Assigned To: Devaraj Das The current strategy of writing a file per target map is consuming a lot of unused buffer space (causing out of memory crashes) and puts a lot of burden on the FS (many opens, inodes used, etc). I propose that we write a single file containing all output and also write an index file IDing which byte range in the file goes to each reduce. This will remove the issue of buffer waste, address scaling issues with number of open files and generally set us up better for scaling. It will also have advantages with very small inputs, since the buffer cache will reduce the number of seeks needed and the data serving node can open a single file and just keep it open rather than needing to do directory and open ops on every request. The only issue I see is that in cases where the task output is substantiallyu larger than its input, we may need to spill multiple times. In this case, we can do a merge after all spills are complete (or during the final spill). -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-489) Seperating user logs from system logs in map reduce
[ http://issues.apache.org/jira/browse/HADOOP-489?page=comments#action_12442249 ] eric baldeschwieler commented on HADOOP-489: [[ Old comment, sent by email on Tue, 10 Oct 2006 09:32:55 -0700 ]] one log file per task does not work if we move the data into DSF. I think we want one log file per running task, but then we need to combine them. Owen? Seperating user logs from system logs in map reduce --- Key: HADOOP-489 URL: http://issues.apache.org/jira/browse/HADOOP-489 Project: Hadoop Issue Type: Improvement Components: mapred Reporter: Mahadev konar Assigned To: Arun C Murthy Priority: Minor Currently the user logs are a part of system logs in mapreduce. Anything logged by the user is logged into the tasktracker log files. This create two issues- 1) The system log files get cluttered with user output. If the user outputs a large amount of logs, the system logs need to be cleaned up pretty often. 2) For the user, it is difficult to get to each of the machines and look for the logs his/her job might have generated. I am proposing three solutions to the problem. All of them have issues with it - Solution 1. Output the user logs on the user screen as part of the job submission process. Merits- This will prevent users from printing large amount of logs and the user can get runtime feedback on what is wrong with his/her job. Issues - This proposal will use the framework bandwidth while running jobs for the user. The user logs will need to pass from the tasks to the tasktrackers, from the tasktrackers to the jobtrackers and then from the jobtrackers to the jobclient using a lot of framework bandwidth if the user is printing out too much data. Solution 2. Output the user logs onto a dfs directory and then concatenate these files. Each task can create a file for the output in the log direcotyr for a given user and jobid. Issues - This will create a huge amount of small files in DFS which later can be concatenated into a single file. Also there is this issue that who would concatenate these files into a single file? This could be done by the framework (jobtracker) as part of the cleanup for the jobs - might stress the jobtracker. Solution 3. Put the user logs into a seperate user log file in the log directory on each tasktrackers. We can provide some tools to query these local log files. We could have commands like for jobid j and for taskid t get me the user log output. These tools could run as a seperate map reduce program with each map grepping the user log files and a single recude aggregating these logs in to a single dfs file. Issues- This does sound like more work for the user. Also, the output might not be complete since a tasktracker might have went down after it ran the job. Any thoughts? -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-591) Reducer sort should even out the pass factors in merging different pass
[ http://issues.apache.org/jira/browse/HADOOP-591?page=comments#action_12441808 ] eric baldeschwieler commented on HADOOP-591: yup Reducer sort should even out the pass factors in merging different pass --- Key: HADOOP-591 URL: http://issues.apache.org/jira/browse/HADOOP-591 Project: Hadoop Issue Type: Improvement Components: mapred Reporter: Runping Qi When multiple pass merging is needed during sort, the current sort implementation in SequenceFile class uses a simple greedy way to select pass factors, resulting uneven pass factor in different passes. For example, if the factor pass is 100 (the default), and there are 101 segments to be merged. The current implementation will first merge the first 100 segments into one and then merge the big output file with the last segment with pass factor 2. It will be better off to use pass factors 11 in the first pass and pass factor 10 in the second pass. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-489) Seperating user logs from system logs in map reduce
[ http://issues.apache.org/jira/browse/HADOOP-489?page=comments#action_12441812 ] eric baldeschwieler commented on HADOOP-489: You need to capture stdout and error. We want that to work. I like the idea of using the logging system if possible to avoid extra code. I don't follow the details of your issues in option #2. I think it is much more important to have a good out of the box experience than to have lots of config file options for changing task logging. Having no user configuration via log4j config files of task logging would probably be fine. Seperating user logs from system logs in map reduce --- Key: HADOOP-489 URL: http://issues.apache.org/jira/browse/HADOOP-489 Project: Hadoop Issue Type: Improvement Components: mapred Reporter: Mahadev konar Assigned To: Arun C Murthy Priority: Minor Currently the user logs are a part of system logs in mapreduce. Anything logged by the user is logged into the tasktracker log files. This create two issues- 1) The system log files get cluttered with user output. If the user outputs a large amount of logs, the system logs need to be cleaned up pretty often. 2) For the user, it is difficult to get to each of the machines and look for the logs his/her job might have generated. I am proposing three solutions to the problem. All of them have issues with it - Solution 1. Output the user logs on the user screen as part of the job submission process. Merits- This will prevent users from printing large amount of logs and the user can get runtime feedback on what is wrong with his/her job. Issues - This proposal will use the framework bandwidth while running jobs for the user. The user logs will need to pass from the tasks to the tasktrackers, from the tasktrackers to the jobtrackers and then from the jobtrackers to the jobclient using a lot of framework bandwidth if the user is printing out too much data. Solution 2. Output the user logs onto a dfs directory and then concatenate these files. Each task can create a file for the output in the log direcotyr for a given user and jobid. Issues - This will create a huge amount of small files in DFS which later can be concatenated into a single file. Also there is this issue that who would concatenate these files into a single file? This could be done by the framework (jobtracker) as part of the cleanup for the jobs - might stress the jobtracker. Solution 3. Put the user logs into a seperate user log file in the log directory on each tasktrackers. We can provide some tools to query these local log files. We could have commands like for jobid j and for taskid t get me the user log output. These tools could run as a seperate map reduce program with each map grepping the user log files and a single recude aggregating these logs in to a single dfs file. Issues- This does sound like more work for the user. Also, the output might not be complete since a tasktracker might have went down after it ran the job. Any thoughts? -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-489) Seperating user logs from system logs in map reduce
[ http://issues.apache.org/jira/browse/HADOOP-489?page=comments#action_12440036 ] eric baldeschwieler commented on HADOOP-489: I don't know exactly what is proposed here. How many files are there, etc. Perhaps you could combine this into a twiki page or an attached spec? Are you writing a file per task? Where? Hours duration on output might work or not. I can imagine cases where you would like to be able to lease these files until they are safely copied into DFS. Might add that. If you store the head of the log, you don't store the failure, that doesn't sound right. Perhaps we can roll the output logs every 25% of target size or so and keep the last 4-5? I like sameer's idea of having a stable jobtracker URL that just redirects to the right task tracker. This would be easy to do with a job in progress, harder with a completed one. Seperating user logs from system logs in map reduce --- Key: HADOOP-489 URL: http://issues.apache.org/jira/browse/HADOOP-489 Project: Hadoop Issue Type: Improvement Components: mapred Reporter: Mahadev konar Assigned To: Owen O'Malley Priority: Minor Currently the user logs are a part of system logs in mapreduce. Anything logged by the user is logged into the tasktracker log files. This create two issues- 1) The system log files get cluttered with user output. If the user outputs a large amount of logs, the system logs need to be cleaned up pretty often. 2) For the user, it is difficult to get to each of the machines and look for the logs his/her job might have generated. I am proposing three solutions to the problem. All of them have issues with it - Solution 1. Output the user logs on the user screen as part of the job submission process. Merits- This will prevent users from printing large amount of logs and the user can get runtime feedback on what is wrong with his/her job. Issues - This proposal will use the framework bandwidth while running jobs for the user. The user logs will need to pass from the tasks to the tasktrackers, from the tasktrackers to the jobtrackers and then from the jobtrackers to the jobclient using a lot of framework bandwidth if the user is printing out too much data. Solution 2. Output the user logs onto a dfs directory and then concatenate these files. Each task can create a file for the output in the log direcotyr for a given user and jobid. Issues - This will create a huge amount of small files in DFS which later can be concatenated into a single file. Also there is this issue that who would concatenate these files into a single file? This could be done by the framework (jobtracker) as part of the cleanup for the jobs - might stress the jobtracker. Solution 3. Put the user logs into a seperate user log file in the log directory on each tasktrackers. We can provide some tools to query these local log files. We could have commands like for jobid j and for taskid t get me the user log output. These tools could run as a seperate map reduce program with each map grepping the user log files and a single recude aggregating these logs in to a single dfs file. Issues- This does sound like more work for the user. Also, the output might not be complete since a tasktracker might have went down after it ran the job. Any thoughts? -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Created: (HADOOP-579) When a HDFS client fails to read a block (due to server failure) the namenode should log this
When a HDFS client fails to read a block (due to server failure) the namenode should log this - Key: HADOOP-579 URL: http://issues.apache.org/jira/browse/HADOOP-579 Project: Hadoop Issue Type: Improvement Components: dfs Reporter: eric baldeschwieler Right now only client debugging info is available. The fact that the client node needed to execute a failure mitigation strategy should be logged centrally so we can do analysis. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Created: (HADOOP-564) we should use hdfs:// in all API URIs
we should use hdfs:// in all API URIs - Key: HADOOP-564 URL: http://issues.apache.org/jira/browse/HADOOP-564 Project: Hadoop Issue Type: Bug Reporter: eric baldeschwieler Minor nit, but it seems that we should choose a protocol name that is likely not to conflict with other distributed FS projects. HDFS seems less likely to. Right now this will be trivial to change. Just wanted to socialize this before doing the search and replace. PS right now the dfs: usage is not consistent and has crept into a couple of new features the y! team is working on (caching of files and distcp). -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Updated: (HADOOP-490) Add ability to send signals to jobs and tasks
[ http://issues.apache.org/jira/browse/HADOOP-490?page=all ] eric baldeschwieler updated HADOOP-490: --- -1 Unless I'm reading this wrong, a file per message would kill the name node at any scale. Also, in a large task, the cost of having every mapper/task scan all the messages could be fairly prohibitive. I'd suggest making it available in contrib or some other mechanism until we see how much uptake it gets. This would leave specific applications free to use it. Perhaps if this gains wide acceptance we could explore moving the concepts into core, but we would need to address the scaling issues to make a general facility. A very interesting set of ideas here, but very complicated if you want to make it work in large general cases. Add ability to send signals to jobs and tasks --- Key: HADOOP-490 URL: http://issues.apache.org/jira/browse/HADOOP-490 Project: Hadoop Issue Type: New Feature Components: mapred Affects Versions: 0.6.0 Reporter: Andrzej Bialecki In some cases it would be useful to be able to signal a job and its tasks about some external condition, or to broadcast a specific message to all tasks in a job. Currently we can only send a single pseudo-signal, that is to kill a job. Example 1: some jobs may be gracefully terminated even if they didn't complete all their work, e.g. Fetcher in Nutch may be running for a very long time if it blocks on relatively few sites left over from the fetchlist. In such case it would be very useful to send it a message requesting that it discards the rest of its input and gracefully completes its map tasks. Example 2: available bandwidth for fetching may be different at different times of day, e.g. daytime vs. nighttime, or total external link usage by other applications. Fetcher jobs often run for several hours. It would be good to be able to send a signal to the Fetcher to throttle or un-throttle its bandwidth usage depending on external conditions. Job implementations could react to these messages either by implementing a method, or by registering a listener, whichever seems more natural. I'm not quite sure how to go about implementing it, I guess this would have to be a part of TaskUmbilicalProtocol but my knowledge here is a bit fuzzy ... ;) Comments are welcome. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Created: (HADOOP-500) Datanode should scan blocks continuously to detect bad blocks / CRC errors
Datanode should scan blocks continuously to detect bad blocks / CRC errors -- Key: HADOOP-500 URL: http://issues.apache.org/jira/browse/HADOOP-500 Project: Hadoop Issue Type: Improvement Reporter: eric baldeschwieler This is a spin-off of the discussion in HADOOP-95 -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-472) Streaming should not crash
[ http://issues.apache.org/jira/browse/HADOOP-472?page=comments#action_12430079 ] eric baldeschwieler commented on HADOOP-472: sounds like checking the reconditions before executing is a good idea. The result of this could just be clearer exceptions. The question is, should this happen? If so, it is not an exception. If not, it is IMO. Streaming should not crash -- Key: HADOOP-472 URL: http://issues.apache.org/jira/browse/HADOOP-472 Project: Hadoop Issue Type: Bug Components: contrib/streaming Reporter: arkady borkovsky Streaming framework should not end with a Java stack dump. All the abnormal conditions should be checked and reported. Specific conditions that need to be exmplicitly checked by the framework are: -- does the input exist? -- do the top level executables or scripts for -mapper and exist Any other Exceptions should also be cought and explained in an error message. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-470) Some improvements in the DFS content browsing UI
[ http://issues.apache.org/jira/browse/HADOOP-470?page=comments#action_12429811 ] eric baldeschwieler commented on HADOOP-470: Recursive size is going to be expensive, right? Don't think we should do that by default. -1 We could provide a link to do a DU to the UI. Some improvements in the DFS content browsing UI Key: HADOOP-470 URL: http://issues.apache.org/jira/browse/HADOOP-470 Project: Hadoop Issue Type: Improvement Components: dfs Reporter: Devaraj Das Priority: Minor Some improvement requests from Yoram: 1. directory browsing: the size, replication and block size fields are unused, and indeed the replication field contains random junk. It would be useful to use these fields to represent the size of the folder (recursive, like du -s), and possibly the number of files in the folder. 2. since file sizes are typically very large, introducing a comma thousands separator will make them more readable. 3. For a particular file I have the list of blocks that make it up. It would be useful to see the block placement information - which datanodes are holding that block. That's arguably more relevant than the block contents when clicking on the block. 4. a nit - 2048 may be too small a chunk size by default. The overhead of getting the first byte is so high (redirect, connect, handshake etc.) that you may as well get 10-20k as your first shot. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-181) task trackers should not restart for having a late heartbeat
[ http://issues.apache.org/jira/browse/HADOOP-181?page=comments#action_12427373 ] eric baldeschwieler commented on HADOOP-181: On reintegrating lost task trackers... It does seem like we should do this to me, but we need to make sure we reason through how this effects corner cases, what invariants the system does maintain and so on. I suggest we work this through, and then go forward with this patch (modified if we find any corner cases) and post the reasoning, so we can review it as this logic evolves. (And update any existing documentation in this area of course...) task trackers should not restart for having a late heartbeat Key: HADOOP-181 URL: http://issues.apache.org/jira/browse/HADOOP-181 Project: Hadoop Issue Type: Bug Components: mapred Reporter: Owen O'Malley Assigned To: Devaraj Das Fix For: 0.6.0 Attachments: lost-heartbeat.patch TaskTrackers should not close and restart themselves for having a late heartbeat. The JobTracker should just accept their current status. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-181) task trackers should not restart for having a late heartbeat
[ http://issues.apache.org/jira/browse/HADOOP-181?page=comments#action_12427052 ] eric baldeschwieler commented on HADOOP-181: Would this have a substantial impact on the large sort benchmark tests our team runs? task trackers should not restart for having a late heartbeat Key: HADOOP-181 URL: http://issues.apache.org/jira/browse/HADOOP-181 Project: Hadoop Issue Type: Bug Components: mapred Reporter: Owen O'Malley Assigned To: Devaraj Das Fix For: 0.6.0 Attachments: lost-heartbeat.patch TaskTrackers should not close and restart themselves for having a late heartbeat. The JobTracker should just accept their current status. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-239) job tracker WI drops jobs after 24 hours
[ http://issues.apache.org/jira/browse/HADOOP-239?page=comments#action_12426324 ] eric baldeschwieler commented on HADOOP-239: +1 to the gist of this (Sanjay's latest suggestions and yoram's point about startup). Putting the log in HDFS is interesting, but perhaps a distraction short term. I think it would be worth trying to use the actual log infrastructure to store this information. Rolling, compression, removal after a fixed time, no lost state when the sever fails... all of this sounds like logging. job tracker WI drops jobs after 24 hours Key: HADOOP-239 URL: http://issues.apache.org/jira/browse/HADOOP-239 Project: Hadoop Issue Type: Bug Components: mapred Reporter: Yoram Arnon Assigned To: Sanjay Dahiya Priority: Minor The jobtracker's WI, keeps track of jobs executed in the past 24 hours. if the cluster was idle for a day (say Sunday) it drops all its history. Monday morning, the page is empty. Better would be to store a fixed number of jobs (say 10 each of succeeded and failed jobs). Also, if the job tracker is restarted, it loses all its history. The history should be persistent, withstanding restarts and upgrades. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-54) SequenceFile should compress blocks, not individual entries
[ http://issues.apache.org/jira/browse/HADOOP-54?page=comments#action_12423105 ] eric baldeschwieler commented on HADOOP-54: --- Folks... I'm happy with everything on this thread now, except for the raw API as discussed. Could folks please consider my suggestion in: http://issues.apache.org/jira/browse/HADOOP-54#action_12422716 I think this addresses all the concerns about sometimes compressed data and avoiding the loose of current functionality etc. I also think that it removes a very dangerous ambiguity that the current owen's version of raw API permit (what is raw? who can use it...). Please let me know what you think of this SequenceFile should compress blocks, not individual entries --- Key: HADOOP-54 URL: http://issues.apache.org/jira/browse/HADOOP-54 Project: Hadoop Issue Type: Improvement Components: io Affects Versions: 0.2.0 Reporter: Doug Cutting Assigned To: Arun C Murthy Fix For: 0.5.0 Attachments: VIntCompressionResults.txt SequenceFile will optionally compress individual values. But both compression and performance would be much better if sequences of keys and values are compressed together. Sync marks should only be placed between blocks. This will require some changes to MapFile too, so that all file positions stored there are the positions of blocks, not entries within blocks. Probably this can be accomplished by adding a getBlockStartPosition() method to SequenceFile.Writer. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-54) SequenceFile should compress blocks, not individual entries
[ http://issues.apache.org/jira/browse/HADOOP-54?page=comments#action_12423218 ] eric baldeschwieler commented on HADOOP-54: --- I completely agree that you should incrementally decompress. The right answer might be just enough for the next entry or a small buffer, should performance test that. My point on raw is that you can return a reference tuple in an object: raw bytes,is compressed flag, compressor class used in a reference Then you read the bytes, decompressed if they come from a block compressed or an uncompressed file, compressed if they come from an item compressed file. Then you pass this reference to the target sequence file's raw write method. The target then compresses or decompresses as needed. Since you package all of this up behind an API, folks will not get confused into using this essentially internal API to do the wrong thing and it will efficiently passed item compressed objects from one such stream to another if given the chance. This may be worth considering, since sorts and merges may often operate on item compressed values and this will avoid a lot of spurious decompression/compression. PS we probably should only bother doing this for values. SequenceFile should compress blocks, not individual entries --- Key: HADOOP-54 URL: http://issues.apache.org/jira/browse/HADOOP-54 Project: Hadoop Issue Type: Improvement Components: io Affects Versions: 0.2.0 Reporter: Doug Cutting Assigned To: Arun C Murthy Fix For: 0.5.0 Attachments: VIntCompressionResults.txt SequenceFile will optionally compress individual values. But both compression and performance would be much better if sequences of keys and values are compressed together. Sync marks should only be placed between blocks. This will require some changes to MapFile too, so that all file positions stored there are the positions of blocks, not entries within blocks. Probably this can be accomplished by adding a getBlockStartPosition() method to SequenceFile.Writer. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (HADOOP-362) tasks can get lost when reporting task completion to the JobTracker has an error
[ http://issues.apache.org/jira/browse/HADOOP-362?page=comments#action_12423221 ] eric baldeschwieler commented on HADOOP-362: +1 to the second idea (unique hashcode / client thread, forcing serialization the server via mapping it to the same thread ID) tasks can get lost when reporting task completion to the JobTracker has an error Key: HADOOP-362 URL: http://issues.apache.org/jira/browse/HADOOP-362 Project: Hadoop Issue Type: Bug Components: mapred Reporter: Devaraj Das Assigned To: Owen O'Malley Attachments: lost-status-updates.patch Basically, the JobTracker used to lose some updates about successful map tasks and it would assume that the tasks are still running (the old progress report is what it used to display in the web page). Now this would cause the reduces to also wait for the map output and they would never receive the output. This would cause the job to appear as if it was hung. The following piece of code sends the status of tasks to the JobTracker: synchronized (this) { for (Iterator it = runningTasks.values().iterator(); it.hasNext(); ) { TaskInProgress tip = (TaskInProgress) it.next(); TaskStatus status = tip.createStatus(); taskReports.add(status); if (status.getRunState() != TaskStatus.RUNNING) { if (tip.getTask().isMapTask()) { mapTotal--; } else { reduceTotal--; } it.remove(); } } } // // Xmit the heartbeat // TaskTrackerStatus status = new TaskTrackerStatus(taskTrackerName, localHostname, httpPort, taskReports, failures); int resultCode = jobClient.emitHeartbeat(status, justStarted); Notice that the completed TIPs are removed from runningTasks data structure. Now, if the emitHeartBeat threw an exception (if it could not communicate with the JobTracker till the IPC timeout expires) then this update is lost. And the next time it sends the hearbeat this completed task's status is missing and hence the JobTracker doesn't know about this completed task. So, one solution to this is to remove the completed TIPs from runningTasks after emitHeartbeat returns. Here is how the new code would look like: synchronized (this) { for (Iterator it = runningTasks.values().iterator(); it.hasNext(); ) { TaskInProgress tip = (TaskInProgress) it.next(); TaskStatus status = tip.createStatus(); taskReports.add(status); } } // // Xmit the heartbeat // TaskTrackerStatus status = new TaskTrackerStatus(taskTrackerName, localHostname, httpPort, taskReports, failures); int resultCode = jobClient.emitHeartbeat(status, justStarted); synchronized (this) { for (Iterator it = runningTasks.values().iterator(); it.hasNext(); ) { TaskInProgress tip = (TaskInProgress) it.next(); if (tip.runstate != TaskStatus.RUNNING) { if (tip.getTask().isMapTask()) { mapTotal--; } else { reduceTotal--; } it.remove(); } } } -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira