You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bigtop.apache.org by rv...@apache.org on 2012/02/04 02:27:07 UTC

svn commit: r1240432 - in /incubator/bigtop/branches/hadoop-0.23: bigtop-deploy/puppet/manifests/ bigtop-deploy/puppet/modules/hadoop/manifests/ bigtop-deploy/puppet/modules/hadoop/templates/ bigtop-packages/src/common/hadoop/ bigtop-packages/src/commo...

Author: rvs
Date: Sat Feb  4 01:27:07 2012
New Revision: 1240432

URL: http://svn.apache.org/viewvc?rev=1240432&view=rev
Log:
BIGTOP-333. heed to start properly packaging httpfs

Added:
    incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/templates/httpfs-env.sh
    incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/templates/httpfs-site.xml
    incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/hadoop-httpfs.default
    incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/service-hadoop-httpfs
    incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.dirs
    incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.install
    incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.postinst
    incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.preinst
    incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.prerm
Modified:
    incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/manifests/cluster.pp
    incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/manifests/init.pp
    incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/conf.pseudo/core-site.xml
    incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/install_hadoop.sh
    incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/control
    incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/rules
    incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/rpm/hadoop/SPECS/hadoop.spec

Modified: incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/manifests/cluster.pp
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/manifests/cluster.pp?rev=1240432&r1=1240431&r2=1240432&view=diff
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/manifests/cluster.pp (original)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/manifests/cluster.pp Sat Feb  4 01:27:07 2012
@@ -42,7 +42,8 @@ class hadoop_cluster_node {
   # $hadoop_mapred_jobtracker_plugins="org.apache.hadoop.thriftfs.ThriftJobTrackerPlugin"
   # $hadoop_mapred_tasktracker_plugins="org.apache.hadoop.mapred.TaskTrackerCmonInst"
 
-  $hadoop_core_proxyusers = { oozie => { groups => 'root,hadoop,jenkins,oozie,users', hosts => "${hadoop_head_node},localhost,127.0.0.1" } }
+  $hadoop_core_proxyusers = { oozie => { groups => 'root,hadoop,jenkins,oozie,users', hosts => "${hadoop_head_node},localhost,127.0.0.1" },
+                             httpfs => { groups => 'root,hadoop,jenkins,oozie,users', hosts => "${hadoop_head_node},localhost,127.0.0.1" } }
 
   $hbase_relative_rootdir        = extlookup("hadoop_hbase_rootdir", "/hbase")
   $hadoop_hbase_rootdir = "hdfs://$hadoop_namenode_host:$hadoop_namenode_port/$hbase_relative_rootdir"
@@ -65,8 +66,9 @@ class hadoop_cluster_node {
     $kerberos_kdc_server = extlookup("hadoop_kerberos_kdc_server")
 
     include kerberos::client
-    kerberos::client::host_keytab { ["hdfs", "yarn", "mapred", "hbase", "oozie"]:
+    kerberos::client::host_keytab { ["hdfs", "httpfs", "yarn", "mapred", "hbase", "oozie"]:
       princs_map => { hdfs   => [ "host", "hdfs" ],
+                      httpfs => [ "httpfs" ],
                       yarn   => [ "yarn"   ],
                       mapred => [ "mapred" ],
                       hbase  => [ "hbase"  ],
@@ -144,6 +146,12 @@ class hadoop_head_node inherits hadoop_c
         auth => $hadoop_security_authentication,
   }
 
+  hadoop::httpfs { "httpfs":
+        namenode_host => $hadoop_namenode_host,
+        namenode_port => $hadoop_namenode_port,
+        auth => $hadoop_security_authentication,
+  }
+
   hadoop-hbase::master { "hbase master":
         rootdir => $hadoop_hbase_rootdir,
         heap_size => $hbase_heap_size,

Modified: incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/manifests/init.pp
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/manifests/init.pp?rev=1240432&r1=1240431&r2=1240432&view=diff
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/manifests/init.pp (original)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/manifests/init.pp Sat Feb  4 01:27:07 2012
@@ -136,6 +136,41 @@ class hadoop {
     }
   }
 
+  define httpfs ($namenode_host, $namenode_port, $port = "14000", $auth = "simple", $secret = "hadoop httpfs secret") {
+
+    $hadoop_namenode_host = $namenode_host
+    $hadoop_namenode_port = $namenode_port
+    $hadoop_httpfs_port = $port
+    $hadoop_security_authentication = $auth
+
+    package { "hadoop-httpfs":
+      ensure => latest,
+      require => Package["jdk"],
+    }
+
+    file { "/etc/hadoop-httpfs/conf/httpfs-site.xml":
+      content => template('hadoop/httpfs-site.xml'),
+      require => [Package["hadoop-hdfs"]],
+    }
+
+    file { "/etc/hadoop-httpfs/conf/httpfs-env.sh":
+      content => template('hadoop/httpfs-env.sh'),
+      require => [Package["hadoop-hdfs"]],
+    }
+
+    file { "/etc/hadoop-httpfs/conf/httpfs-signature.secret":
+      content => inline_template("<%= secret %>"),
+      require => [Package["hadoop-hdfs"]],
+    }
+
+    service { "hadoop-httpfs":
+      ensure => running,
+      hasstatus => true,
+      subscribe => [Package["hadoop-httpfs"], File["/etc/hadoop-httpfs/conf/httpfs-site.xml"], File["/etc/hadoop-httpfs/conf/httpfs-env.sh"], File["/etc/hadoop-httpfs/conf/httpfs-signature.secret"]],
+      require => [ Package["hadoop-httpfs"] ],
+    }
+  }
+
   define create_hdfs_dirs($hdfs_dirs_meta) {
     $user = $hdfs_dirs_meta[$title][user]
     $perm = $hdfs_dirs_meta[$title][perm]

Added: incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/templates/httpfs-env.sh
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/templates/httpfs-env.sh?rev=1240432&view=auto
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/templates/httpfs-env.sh (added)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/templates/httpfs-env.sh Sat Feb  4 01:27:07 2012
@@ -0,0 +1,41 @@
+#!/bin/bash
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License. See accompanying LICENSE file.
+#
+
+# Set httpfs specific environment variables here.
+
+# Settings for the Embedded Tomcat that runs HttpFS
+# Java System properties for HttpFS should be specified in this variable
+#
+# export CATALINA_OPTS=
+
+# HttpFS logs directory
+#
+# export HTTPFS_LOG=${HTTPFS_HOME}/logs
+
+# HttpFS temporary directory
+#
+# export HTTPFS_TEMP=${HTTPFS_HOME}/temp
+
+# The HTTP port used by HttpFS
+#
+export HTTPFS_HTTP_PORT=<%= hadoop_httpfs_port %>
+
+# The Admin port used by HttpFS
+#
+# export HTTPFS_ADMIN_PORT=`expr ${HTTPFS_HTTP_PORT} + 1`
+
+# The hostname HttpFS server runs on
+#
+# export HTTPFS_HTTP_HOSTNAME=`hostname -f`

Added: incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/templates/httpfs-site.xml
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/templates/httpfs-site.xml?rev=1240432&view=auto
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/templates/httpfs-site.xml (added)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-deploy/puppet/modules/hadoop/templates/httpfs-site.xml Sat Feb  4 01:27:07 2012
@@ -0,0 +1,61 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Licensed to the Apache Software Foundation (ASF) under one or more       -->
+<!-- contributor license agreements.  See the NOTICE file distributed with    -->
+<!-- this work for additional information regarding copyright ownership.      -->
+<!-- The ASF licenses this file to You under the Apache License, Version 2.0  -->
+<!-- (the "License"); you may not use this file except in compliance with     -->
+<!-- the License.  You may obtain a copy of the License at                    -->
+<!--                                                                          -->
+<!--     http://www.apache.org/licenses/LICENSE-2.0                           -->
+<!--                                                                          -->
+<!-- Unless required by applicable law or agreed to in writing, software      -->
+<!-- distributed under the License is distributed on an "AS IS" BASIS,        -->
+<!-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -->
+<!-- See the License for the specific language governing permissions and      -->
+<!-- limitations under the License.                                           -->
+
+<configuration>
+
+<% if hadoop_security_authentication == "kerberos" %>
+  <property>
+    <name>httpfs.authentication.type</name>
+    <value>kerberos</value>
+  </property>
+  <property>
+    <name>kerberos.realm</name>
+    <value><%= kerberos_realm %></value>
+  </property>
+  <property>
+    <name>httpfs.authentication.kerberos.principal</name>
+    <value>httpfs/<%= fqdn %></value>
+  </property>
+  <property>
+    <name>httpfs.authentication.kerberos.keytab</name>
+    <value>/etc/httpfs.keytab</value>
+  </property>
+
+  <property>
+    <name>httpfs.hadoop.authentication.type</name>
+    <value>kerberos</value>
+  </property>
+  <property>
+    <name>httpfs.hadoop.authentication.kerberos.principal</name>
+    <value>${httpfs.authentication.kerberos.principal}</value>
+  </property>
+  <property>
+    <name>httpfs.hadoop.authentication.kerberos.keytab</name>
+    <value>${httpfs.authentication.kerberos.keytab}</value>
+  </property>
+  <property>
+    <name>httpfs.hadoop.conf:dfs.namenode.kerberos.principal</name>
+    <value>hdfs/<%= hadoop_namenode_host %>@<%= kerberos_realm %></value>
+  </property>
+<% end %>
+
+  <property>
+    <name>httpfs.hadoop.conf:fs.default.name</name>
+    <value>hdfs://<%= hadoop_namenode_host %>:<%= hadoop_namenode_port%></value>
+  </property>
+</configuration>

Modified: incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/conf.pseudo/core-site.xml
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/conf.pseudo/core-site.xml?rev=1240432&r1=1240431&r2=1240432&view=diff
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/conf.pseudo/core-site.xml (original)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/conf.pseudo/core-site.xml Sat Feb  4 01:27:07 2012
@@ -38,4 +38,14 @@
     <value>*</value>
   </property>
 
+  <!-- HTTPFS proxy user setting -->
+  <property>
+    <name>hadoop.proxyuser.httpfs.hosts</name>
+    <value>*</value>
+  </property>
+  <property>
+    <name>hadoop.proxyuser.httpfs.groups</name>
+    <value>*</value>
+  </property>
+
 </configuration>

Added: incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/hadoop-httpfs.default
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/hadoop-httpfs.default?rev=1240432&view=auto
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/hadoop-httpfs.default (added)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/hadoop-httpfs.default Sat Feb  4 01:27:07 2012
@@ -0,0 +1,23 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+export HTTPFS_USER=httpfs
+export HTTPFS_CONFIG=/etc/hadoop-httpfs/conf
+export HTTPFS_LOG=/var/log/hadoop-httpfs/
+export HTTPFS_TEMP=/var/run/hadoop-httpfs/
+export CATALINA_PID=/var/run/hadoop-httpfs/hadoop-httpfs.pid
+export CATALINA_BASE=/usr/lib/hadoop-httpfs
+export CATALINA_TMPDIR=/var/run/hadoop-httpfs/
+# HTTPFS_HTTP_PORT
+# HTTPFS_ADMIN_PORT

Modified: incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/install_hadoop.sh
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/install_hadoop.sh?rev=1240432&r1=1240431&r2=1240432&view=diff
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/install_hadoop.sh (original)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/install_hadoop.sh Sat Feb  4 01:27:07 2012
@@ -40,10 +40,12 @@ OPTS=$(getopt \
   -l 'native-build-string:' \
   -l 'installed-lib-dir:' \
   -l 'hadoop-dir:' \
+  -l 'httpfs-dir:' \
   -l 'system-include-dir:' \
   -l 'system-lib-dir:' \
   -l 'system-libexec-dir:' \
   -l 'hadoop-etc-dir:' \
+  -l 'httpfs-etc-dir:' \
   -l 'doc-dir:' \
   -l 'man-dir:' \
   -l 'example-dir:' \
@@ -63,6 +65,9 @@ while true ; do
         --distro-dir)
         DISTRO_DIR=$2 ; shift 2
         ;;
+        --httpfs-dir)
+        HTTPFS_DIR=$2 ; shift 2
+        ;;
         --hadoop-dir)
         HADOOP_DIR=$2 ; shift 2
         ;;
@@ -87,6 +92,9 @@ while true ; do
         --hadoop-etc-dir)
         HADOOP_ETC_DIR=$2 ; shift 2
         ;;
+        --httpfs-etc-dir)
+        HTTPFS_ETC_DIR=$2 ; shift 2
+        ;;
         --installed-lib-dir)
         INSTALLED_LIB_DIR=$2 ; shift 2
         ;;
@@ -115,6 +123,7 @@ for var in PREFIX BUILD_DIR; do
 done
 
 HADOOP_DIR=${HADOOP_DIR:-$PREFIX/usr/lib/hadoop}
+HTTPFS_DIR=${HTTPFS_DIR:-$PREFIX/usr/lib/hadoop-httpfs}
 SYSTEM_LIB_DIR=${SYSTEM_LIB_DIR:-/usr/lib}
 BIN_DIR=${BIN_DIR:-$PREFIX/usr/bin}
 DOC_DIR=${DOC_DIR:-$PREFIX/usr/share/doc/hadoop}
@@ -123,6 +132,7 @@ SYSTEM_INCLUDE_DIR=${SYSTEM_INCLUDE_DIR:
 SYSTEM_LIBEXEC_DIR=${SYSTEM_LIBEXEC_DIR:-$PREFIX/usr/libexec}
 EXAMPLE_DIR=${EXAMPLE_DIR:-$DOC_DIR/examples}
 HADOOP_ETC_DIR=${HADOOP_ETC_DIR:-$PREFIX/etc/hadoop}
+HTTPFS_ETC_DIR=${HTTPFS_ETC_DIR:-$PREFIX/etc/hadoop-httpfs}
 
 INSTALLED_HADOOP_DIR=${INSTALLED_HADOOP_DIR:-/usr/lib/hadoop}
 
@@ -175,6 +185,16 @@ cp -a ${BUILD_DIR}/bin/* ${HADOOP_BIN_DI
 install -d -m 0755 ${HADOOP_SBIN_DIR}
 cp ${BUILD_DIR}/sbin/* ${HADOOP_SBIN_DIR}/
 
+# HTTPFS
+install -d -m 0755 ${HTTPFS_DIR}/sbin
+cp ${HADOOP_SBIN_DIR}/httpfs.sh ${HTTPFS_DIR}/sbin/
+install -d -m 0755 ${HTTPFS_DIR}/libexec
+cp ${SYSTEM_LIBEXEC_DIR}/httpfs-config.sh ${HTTPFS_DIR}/libexec/
+cp -r ${BUILD_DIR}/share/hadoop/httpfs/tomcat/* ${HTTPFS_DIR}/
+chmod 644 ${HTTPFS_DIR}/conf/*
+install -d -m 0755 $HTTPFS_ETC_DIR/conf.empty
+cp ${BUILD_DIR}/etc/hadoop/httpfs* $HTTPFS_ETC_DIR/conf.empty
+
 # jars
 install -d -m 0755 ${HADOOP_LIB_DIR}
 cp ${BUILD_DIR}/share/hadoop/mapreduce/lib/*.jar ${HADOOP_LIB_DIR}/

Added: incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/service-hadoop-httpfs
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/service-hadoop-httpfs?rev=1240432&view=auto
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/service-hadoop-httpfs (added)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/common/hadoop/service-hadoop-httpfs Sat Feb  4 01:27:07 2012
@@ -0,0 +1,145 @@
+#!/bin/bash
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+# Starts a Hadoop HTTPFS daemon
+#
+# chkconfig: 345 90 10
+# description: Hadoop HTTPFS daemon
+#
+### BEGIN INIT INFO
+# Provides:          hadoop-httpfs
+# Required-Start:    $remote_fs
+# Should-Start:
+# Required-Stop:     $remote_fs
+# Should-Stop:
+# Default-Start:     3 4 5
+# Default-Stop:      0 1 2 6
+# Short-Description: Hadoop HTTPFS daemon
+### END INIT INFO
+
+. /lib/lsb/init-functions
+. /etc/default/hadoop-httpfs
+
+# Autodetect JAVA_HOME if not defined
+if [ -e /usr/libexec/bigtop-detect-javahome ]; then
+  . /usr/libexec/bigtop-detect-javahome
+elif [ -e /usr/lib/bigtop-utils/bigtop-detect-javahome ]; then
+  . /usr/lib/bigtop-utils/bigtop-detect-javahome
+fi
+
+STATUS_RUNNING=0
+STATUS_DEAD=1
+STATUS_DEAD_AND_LOCK=2
+STATUS_NOT_RUNNING=3
+
+ERROR_PROGRAM_NOT_INSTALLED=5
+
+export HTTPFS_USER=${HTTPFS_USER:-"httpfs"}
+export HTTPFS_CONFIG=${HTTPFS_CONFIG:-"/etc/hadoop-httpfs/conf"}
+export HTTPFS_LOG=${HTTPFS_LOG:-"/var/log/hadoop-httpfs/"}
+export HTTPFS_TEMP=${HTTPFS_TEMP:-"/var/run/hadoop-httpfs/"}
+export HTTPFS_SLEEP_TIME=${HTTPFS_SLEEP_TIME:-"5"}
+export CATALINA_PID=${CATALINA_PID:-"/var/run/hadoop-httpfs/hadoop-httpfs.pid"}
+export CATALINA_BASE=${CATALINA_BASE:-"/usr/lib/hadoop-httpfs"}
+export CATALINA_TMPDIR=${CATALINA_TMPDIR:-"/var/run/hadoop-httpfs/"}
+
+desc="Hadoop HTTPFS daemon"
+EXEC_PATH="$CATALINA_BASE/sbin/httpfs.sh"
+LOCKFILE="/var/lock/subsys/hadoop-httpfs"
+
+# Support ephemeral /var/run (don't bail -- just silently try our best) 
+install -d -m 0775 -o $HTTPFS_USER -g $HTTPFS_USER "$(dirname "$CATALINA_PID")" 2>/dev/null || :
+
+start() {
+  [ -x "$EXEC_PATH" ] || exit $ERROR_PROGRAM_NOT_INSTALLED
+  log_success_msg "Starting ${desc}: "
+  /bin/su -s /bin/bash -c "${EXEC_PATH} start $DAEMON_FLAGS" $HTTPFS_USER
+  RETVAL=$?
+  [ $RETVAL -eq 0 ] && touch $LOCKFILE
+  return $RETVAL
+}
+
+stop() {
+  if [ ! -e $CATALINA_PID ]; then
+    log_failure_msg "$desc is not running"
+    RETVAL=$STATUS_NOT_RUNNING
+    return $RETVAL
+  fi
+
+  log_success_msg "Stopping ${desc}: "
+
+  su -s /bin/bash $HTTPFS_USER -c "${EXEC_PATH} stop $HTTPFS_SLEEP_TIME -force"
+  RETVAL=$?
+  [ $RETVAL -eq 0 ] && rm -f $LOCKFILE 2>/dev/null
+  return $RETVAL
+}
+
+restart() {
+  stop
+  start
+}
+
+checkstatus(){
+  pidofproc -p $CATALINA_PID java > /dev/null
+  status=$?
+
+  case "$status" in
+    $STATUS_RUNNING)
+      log_success_msg "$desc is running"
+      ;;
+    $STATUS_DEAD)
+      log_failure_msg "$desc is dead and pid file exists"
+      ;;
+    $STATUS_DEAD_AND_LOCK)
+      log_failure_msg "$desc is dead and lock file exists"
+      ;;
+    $STATUS_NOT_RUNNING)
+      log_failure_msg "$desc is not running"
+      ;;
+    *)
+      log_failure_msg "$desc status is unknown"
+      ;;
+  esac
+  return $status
+}
+
+condrestart(){
+  [ -e $LOCKFILE ] && restart || :
+}
+
+case "$1" in
+  start)
+    start
+    ;;
+  stop)
+    stop
+    ;;
+  status)
+    checkstatus
+    ;;
+  restart)
+    restart
+    ;;
+  condrestart|try-restart)
+    condrestart
+    ;;
+  *)
+    echo $"Usage: $0 {start|stop|status|restart|try-restart|condrestart}"
+    exit 1
+esac
+
+exit $RETVAL

Modified: incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/control
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/control?rev=1240432&r1=1240431&r2=1240432&view=diff
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/control (original)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/control Sat Feb  4 01:27:07 2012
@@ -185,3 +185,11 @@ Section: libdevel
 Depends: hadoop (= ${source:Version}), libhdfs0 (= ${binary:Version})
 Description: Development support for libhdfs0
  Includes examples and header files for accessing HDFS from C
+
+Package: hadoop-httpfs
+Provides: hadoop-httpfs
+Architecture: all
+Depends: hadoop-hdfs (= ${source:Version})
+Description: HTTPFS for Hadoop
+  The server providing HTTP REST API support for the complete FileSystem/FileContext
+  interface in HDFS.

Added: incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.dirs
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.dirs?rev=1240432&view=auto
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.dirs (added)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.dirs Sat Feb  4 01:27:07 2012
@@ -0,0 +1,4 @@
+/etc/hadoop-httpfs
+/usr/lib/hadoop-httpfs
+/var/log/hadoop-httpfs
+/var/run/hadoop-httpfs

Added: incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.install
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.install?rev=1240432&view=auto
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.install (added)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.install Sat Feb  4 01:27:07 2012
@@ -0,0 +1,3 @@
+/etc/default/hadoop-httpfs
+/etc/hadoop-httpfs
+/usr/lib/hadoop-httpfs

Added: incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.postinst
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.postinst?rev=1240432&view=auto
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.postinst (added)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.postinst Sat Feb  4 01:27:07 2012
@@ -0,0 +1,37 @@
+#!/bin/sh
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# postinst script for hadoop
+
+set -e
+
+case "$1" in
+    configure)
+        chown httpfs:httpfs /var/run/hadoop-httpfs /var/log/hadoop-httpfs
+        update-alternatives --install /etc/hadoop-httpfs/conf hadoop-httpfs-conf /etc/hadoop-httpfs/conf.empty 10
+    ;;
+
+    abort-upgrade|abort-remove|abort-deconfigure)
+    ;;
+
+    *)
+        echo "postinst called with unknown argument \`$1'" >&2
+        exit 1
+    ;;
+esac
+
+#DEBHELPER#

Added: incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.preinst
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.preinst?rev=1240432&view=auto
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.preinst (added)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.preinst Sat Feb  4 01:27:07 2012
@@ -0,0 +1,60 @@
+#!/bin/sh
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# preinst script for hadoop-httpfs
+#
+# see: dh_installdeb(1)
+
+set -e
+
+# summary of how this script can be called:
+#        * <new-preinst> `install'
+#        * <new-preinst> `install' <old-version>
+#        * <new-preinst> `upgrade' <old-version>
+#        * <old-preinst> `abort-upgrade' <new-version>
+# for details, see http://www.debian.org/doc/debian-policy/ or
+# the debian-policy package
+
+
+case "$1" in
+    install|upgrade)
+        if ! getent passwd httpfs >/dev/null; then
+            adduser \
+                --system \
+                --group \
+                --home /var/run/hadoop-httpfs \
+                --gecos "Hadoop HTTPFS" \
+                --shell /bin/bash \
+                httpfs >/dev/null 2>/dev/null || :
+        fi
+    ;;
+
+    abort-upgrade)
+    ;;
+
+    *)
+        echo "preinst called with unknown argument \`$1'" >&2
+        exit 1
+    ;;
+esac
+
+# dh_installdeb will replace this with shell code automatically
+# generated by other debhelper scripts.
+
+#DEBHELPER#
+
+exit 0

Added: incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.prerm
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.prerm?rev=1240432&view=auto
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.prerm (added)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/hadoop-httpfs.prerm Sat Feb  4 01:27:07 2012
@@ -0,0 +1,57 @@
+#!/bin/sh
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# prerm script for hadoop-httpfs
+#
+# see: dh_installdeb(1)
+
+set -e
+
+# summary of how this script can be called:
+#        * <prerm> `remove'
+#        * <old-prerm> `upgrade' <new-version>
+#        * <new-prerm> `failed-upgrade' <old-version>
+#        * <conflictor's-prerm> `remove' `in-favour' <package> <new-version>
+#        * <deconfigured's-prerm> `deconfigure' `in-favour'
+#          <package-being-installed> <version> `removing'
+#          <conflicting-package> <version>
+# for details, see http://www.debian.org/doc/debian-policy/ or
+# the debian-policy package
+
+
+case "$1" in
+    remove|upgrade|deconfigure)
+      update-alternatives --remove hadoop-httpfs-conf /etc/hadoop-httpfs/conf.empty || :
+    ;;
+
+    failed-upgrade)
+    ;;
+
+    *)
+        echo "prerm called with unknown argument \`$1'" >&2
+        exit 1
+    ;;
+esac
+
+# dh_installdeb will replace this with shell code automatically
+# generated by other debhelper scripts.
+
+#DEBHELPER#
+
+exit 0
+
+

Modified: incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/rules
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/rules?rev=1240432&r1=1240431&r2=1240432&view=diff
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/rules (original)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/deb/hadoop/rules Sat Feb  4 01:27:07 2012
@@ -61,6 +61,8 @@ install: build
 	  --prefix=debian/tmp/ \
 	  --distro-dir=debian \
 	  --build-dir=${PWD}/build \
+	  --httpfs-dir=debian/tmp/usr/lib/hadoop-httpfs \
+	  --httpfs-etc-dir=debian/tmp/etc/hadoop-httpfs \
 	  --system-lib-dir=debian/tmp/usr/lib/ \
 	  --system-libexec-dir=debian/tmp/usr/lib/hadoop/libexec/ \
 	  --system-include-dir=debian/tmp/usr/include \
@@ -80,6 +82,9 @@ datanode_user=hdfs
 resourcemanager_user=yarn
 nodemanager_user=yarn
 
+httpfs: debian/service-hadoop-httpfs
+	cp $< debian/hadoop-$@.init
+
 hdfs-namenode hdfs-secondarynamenode hdfs-datanode mapreduce-historyserver: debian/service-init.d.tpl
 	sed -e "s|@HADOOP_DAEMON@|$(patsubst mapreduce-%,%,$(patsubst hdfs-%,%,$@))|" \
             -e "s|@HADOOP_MAJOR_VERSION@|$hadoop_version|" \
@@ -95,10 +100,11 @@ yarn-resourcemanager yarn-nodemanager: d
 	sed -e "s|@HADOOP_DAEMON@|$@|" < debian/hadoop.daemon.postinst.tpl > debian/hadoop-$@.postinst
 
 
-install-indep: hdfs-namenode hdfs-secondarynamenode hdfs-datanode yarn-resourcemanager yarn-nodemanager mapreduce-historyserver
+install-indep: hdfs-namenode hdfs-secondarynamenode hdfs-datanode yarn-resourcemanager yarn-nodemanager mapreduce-historyserver httpfs
 	mkdir -p debian/tmp/etc/default
 	cp debian/hadoop.default debian/tmp/etc/default/hadoop
 	cp debian/yarn.default debian/tmp/etc/default/yarn
+	cp debian/hadoop-httpfs.default debian/tmp/etc/default/hadoop-httpfs
 
 	dh_install --sourcedir=debian/tmp -i
 

Modified: incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/rpm/hadoop/SPECS/hadoop.spec
URL: http://svn.apache.org/viewvc/incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/rpm/hadoop/SPECS/hadoop.spec?rev=1240432&r1=1240431&r2=1240432&view=diff
==============================================================================
--- incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/rpm/hadoop/SPECS/hadoop.spec (original)
+++ incubator/bigtop/branches/hadoop-0.23/bigtop-packages/src/rpm/hadoop/SPECS/hadoop.spec Sat Feb  4 01:27:07 2012
@@ -24,19 +24,24 @@
 %define hadoop_name hadoop
 %define etc_hadoop /etc/%{name}
 %define etc_yarn /etc/yarn
+%define etc_httpfs /etc/%{name}-httpfs
 %define config_hadoop %{etc_hadoop}/conf
 %define config_yarn %{etc_yarn}/conf
+%define config_httpfs %{etc_httpfs}/conf
 %define lib_hadoop_dirname /usr/lib
 %define lib_hadoop %{lib_hadoop_dirname}/%{name}
+%define lib_httpfs %{lib_hadoop_dirname}/%{name}-httpfs
 %define log_hadoop_dirname /var/log
 %define log_hadoop %{log_hadoop_dirname}/%{name}
 %define log_yarn %{log_hadoop_dirname}/yarn
 %define log_hdfs %{log_hadoop_dirname}/hdfs
+%define log_httpfs %{log_hadoop_dirname}/%{name}-httpfs
 %define log_mapreduce %{log_hadoop_dirname}/mapreduce
 %define run_hadoop_dirname /var/run
 %define run_hadoop %{run_hadoop_dirname}/hadoop
 %define run_yarn %{run_hadoop_dirname}/yarn
 %define run_hdfs %{run_hadoop_dirname}/hdfs
+%define run_httpfs %{run_hadoop_dirname}/%{name}-httpfs
 %define run_mapreduce %{run_hadoop_dirname}/mapreduce
 %define state_hadoop_dirname /var/lib
 %define state_hadoop %{state_hadoop_dirname}/hadoop
@@ -139,6 +144,8 @@ Source7: hadoop-fuse-dfs.1
 Source8: hadoop-fuse.default
 Source9: hadoop.nofiles.conf
 Source10: yarn-init.tmpl
+Source11: hadoop-httpfs.default
+Source12: service-hadoop-httpfs
 Patch0: MAPREDUCE-3436_rev2.patch
 Patch1: MAPREDUCE-3791.patch
 Buildroot: %{_tmppath}/%{name}-%{version}-%{release}-root-%(%{__id} -u -n)
@@ -267,6 +274,16 @@ The Data Nodes in the Hadoop Cluster are
 blocks of data over the network to Hadoop Distributed Filesystem
 (HDFS) clients.
 
+%package httpfs
+Summary: HTTPFS for Hadoop
+Group: System/Daemons
+Requires: %{name}-hdfs = %{version}-%{release}
+Requires(pre): %{name} = %{version}-%{release}
+
+%description httpfs
+The server providing HTTP REST API support for the complete FileSystem/FileContext
+interface in HDFS.
+
 %package yarn-resourcemanager
 Summary: Yarn Resource Manager
 Group: System/Daemons
@@ -347,10 +364,12 @@ env HADOOP_VERSION=%{hadoop_base_version
 bash %{SOURCE2} \
   --distro-dir=$RPM_SOURCE_DIR \
   --build-dir=$PWD/build \
+  --httpfs-dir=$RPM_BUILD_ROOT%{lib_httpfs} \
   --system-include-dir=$RPM_BUILD_ROOT%{_includedir} \
   --system-lib-dir=$RPM_BUILD_ROOT%{_libdir} \
   --system-libexec-dir=$RPM_BUILD_ROOT/%{lib_hadoop}/libexec \
   --hadoop-etc-dir=$RPM_BUILD_ROOT%{etc_hadoop} \
+  --httpfs-etc-dir=$RPM_BUILD_ROOT%{etc_httpfs} \
   --prefix=$RPM_BUILD_ROOT \
   --doc-dir=$RPM_BUILD_ROOT%{doc_hadoop} \
   --example-dir=$RPM_BUILD_ROOT%{doc_hadoop}/examples \
@@ -369,6 +388,7 @@ orig_init_file=$RPM_SOURCE_DIR/hadoop-in
 %endif
 
 yarn_orig_init_file=$RPM_SOURCE_DIR/yarn-init.tmpl
+httpfs_orig_init_file=$RPM_SOURCE_DIR/service-hadoop-httpfs
 
 # Generate the init.d scripts
 for service in %{hdfs_services} %{mapreduce_services}
@@ -391,12 +411,15 @@ do
        %__sed -i -e 's|@YARN_DAEMON_USER@|yarn|' $init_file
        chmod 755 $init_file
 done
+%__cp $httpfs_orig_init_file $RPM_BUILD_ROOT/%{initd_dir}/%{name}-httpfs
+chmod 755 $RPM_BUILD_ROOT/%{initd_dir}/%{name}-httpfs
 
 
 %__install -d -m 0755 $RPM_BUILD_ROOT/etc/default
 %__cp $RPM_SOURCE_DIR/hadoop.default $RPM_BUILD_ROOT/etc/default/hadoop
 %__cp $RPM_SOURCE_DIR/yarn.default $RPM_BUILD_ROOT/etc/default/yarn
-%__cp $RPM_SOURCE_DIR/hadoop-fuse.default $RPM_BUILD_ROOT/etc/default/hadoop-fuse
+%__cp $RPM_SOURCE_DIR/%{name}-fuse.default $RPM_BUILD_ROOT/etc/default/%{name}-fuse
+%__cp $RPM_SOURCE_DIR/%{name}-httpfs.default $RPM_BUILD_ROOT/etc/default/%{name}-httpfs
 
 %__install -d -m 0755 $RPM_BUILD_ROOT/etc/security/limits.d
 %__install -m 0644 %{SOURCE9} $RPM_BUILD_ROOT/etc/security/limits.d/hadoop.nofiles.conf
@@ -411,11 +434,13 @@ done
 %__install -d -m 0775 $RPM_BUILD_ROOT/%{log_yarn}
 # %__install -d -m 0775 $RPM_BUILD_ROOT/%{log_hdfs}
 # %__install -d -m 0775 $RPM_BUILD_ROOT/%{log_mapreduce}
+%__install -d -m 0775 $RPM_BUILD_ROOT/%{log_httpfs}
 # /var/run/*
 %__install -d -m 0775 $RPM_BUILD_ROOT/%{run_hadoop}
 %__install -d -m 0775 $RPM_BUILD_ROOT/%{run_yarn}
 #%__install -d -m 0775 $RPM_BUILD_ROOT/%{run_hdfs}
 #%__install -d -m 0775 $RPM_BUILD_ROOT/%{run_mapreduce}
+%__install -d -m 0775 $RPM_BUILD_ROOT/%{run_httpfs}
 
 %pre
 getent group hadoop >/dev/null || groupadd -r hadoop
@@ -424,6 +449,10 @@ getent group hadoop >/dev/null || groupa
 getent group hdfs >/dev/null   || groupadd -r hdfs
 getent passwd hdfs >/dev/null || /usr/sbin/useradd --comment "Hadoop HDFS" --shell /bin/bash -M -r -g hdfs -G hadoop --home %{state_hdfs} hdfs
 
+%pre httpfs 
+getent group httpfs >/dev/null   || groupadd -r httpfs
+getent passwd httpfs >/dev/null || /usr/sbin/useradd --comment "Hadoop HTTPFS" --shell /bin/bash -M -r -g httpfs -G httpfs --home %{run_httpfs} httpfs
+
 %pre yarn
 getent group yarn >/dev/null   || groupadd -r yarn
 getent passwd yarn >/dev/null || /usr/sbin/useradd --comment "Hadoop Yarn" --shell /bin/bash -M -r -g yarn -G hadoop --home %{state_yarn} yarn
@@ -445,6 +474,9 @@ touch %{log_hadoop}/SecurityAuth.audit
 chgrp hadoop %{log_hadoop}/SecurityAuth.audit
 chmod g+w %{log_hadoop}/SecurityAuth.audit
 
+%post httpfs
+%{alternatives_cmd} --install %{config_httpfs} %{name}-httpfs-conf %{etc_httpfs}/conf.empty 10
+chkconfig --add %{name}-httpfs
 
 %preun
 if [ "$1" = 0 ]; then
@@ -457,6 +489,18 @@ if [ "$1" = 0 ]; then
   %{alternatives_cmd} --remove %{hadoop_name}-default %{bin_hadoop}/%{name} || :
 fi
 
+%preun httpfs
+%{alternatives_cmd} --remove %{name}-httpfs-conf %{etc_httpfs}/conf.empty 10
+if [ $1 = 0 ]; then
+  service %{name}-httpfs stop > /dev/null 2>&1
+  chkconfig --del %{name}-httpfs
+fi
+
+%postun httpfs
+if [ $1 -ge 1 ]; then
+  service %{name}-httpfs condrestart >/dev/null 2>&1
+fi
+
 
 %files yarn
 %defattr(-,root,root)
@@ -481,12 +525,10 @@ fi
 %files hdfs
 %defattr(-,root,root)
 %config(noreplace) %{etc_hadoop}/conf.empty/hdfs-site.xml
-%config(noreplace) %{etc_hadoop}/conf.empty/httpfs-*
 %config(noreplace) /etc/default/hadoop-fuse
 %{lib_hadoop}/hadoop-hdfs*.jar
 %{lib_hadoop}/hadoop-archives*.jar
 %{lib_hadoop}/libexec/hdfs-config.sh
-%{lib_hadoop}/libexec/httpfs-config.sh
 %{lib_hadoop}/libexec/jsvc
 %{lib_hadoop}/webapps
 %{lib_hadoop}/sbin/update-hdfs-env.sh
@@ -498,7 +540,6 @@ fi
 %{lib_hadoop}/sbin/stop-dfs.sh
 %{lib_hadoop}/sbin/refresh-namenodes.sh
 %{lib_hadoop}/sbin/distribute-exclude.sh
-%{lib_hadoop}/sbin/httpfs.sh
 %{lib_hadoop}/bin/hdfs
 %{bin_hadoop}/hdfs
 %attr(0775,hdfs,hadoop) %{run_hdfs}
@@ -553,6 +594,14 @@ fi
 %defattr(-,root,root)
 %doc %{doc_hadoop}
 
+%files httpfs
+%defattr(-,root,root)
+%config(noreplace) %{etc_httpfs}/conf.empty
+%config(noreplace) /etc/default/%{name}-httpfs
+%{initd_dir}/%{name}-%1
+%{lib_httpfs}
+%attr(0775,httpfs,httpfs) %{run_httpfs}
+%attr(0775,httpfs,httpfs) %{log_httpfs}
 
 # Service file management RPMs
 %define service_macro() \