GUACAMOLE-200: Refactor RDPDR printer such that the "ack" handler cannot block.


Project: http://git-wip-us.apache.org/repos/asf/incubator-guacamole-server/repo
Commit: 
http://git-wip-us.apache.org/repos/asf/incubator-guacamole-server/commit/3fc43fba
Tree: 
http://git-wip-us.apache.org/repos/asf/incubator-guacamole-server/tree/3fc43fba
Diff: 
http://git-wip-us.apache.org/repos/asf/incubator-guacamole-server/diff/3fc43fba

Branch: refs/heads/master
Commit: 3fc43fba37043cb1f00bde0cccbf194c87a8dbc7
Parents: 4da4ce7
Author: Michael Jumper <mjum...@apache.org>
Authored: Mon Feb 13 14:44:06 2017 -0800
Committer: Michael Jumper <mjum...@apache.org>
Committed: Mon Feb 13 18:42:28 2017 -0800

----------------------------------------------------------------------
 src/protocols/rdp/Makefile.am                  |   2 +
 src/protocols/rdp/guac_rdpdr/rdpdr_print_job.c | 581 ++++++++++++++++++++
 src/protocols/rdp/guac_rdpdr/rdpdr_print_job.h | 226 ++++++++
 src/protocols/rdp/guac_rdpdr/rdpdr_printer.c   | 359 ++----------
 src/protocols/rdp/guac_rdpdr/rdpdr_printer.h   |  31 --
 5 files changed, 848 insertions(+), 351 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-guacamole-server/blob/3fc43fba/src/protocols/rdp/Makefile.am
----------------------------------------------------------------------
diff --git a/src/protocols/rdp/Makefile.am b/src/protocols/rdp/Makefile.am
index a190003..7daf0e6 100644
--- a/src/protocols/rdp/Makefile.am
+++ b/src/protocols/rdp/Makefile.am
@@ -72,6 +72,7 @@ guacdr_sources =                             \
     guac_rdpdr/rdpdr_fs_service.c            \
     guac_rdpdr/rdpdr_messages.c              \
     guac_rdpdr/rdpdr_printer.c               \
+    guac_rdpdr/rdpdr_print_job.c             \
     guac_rdpdr/rdpdr_service.c               \
     rdp_fs.c                                 \
     rdp_stream.c                             \
@@ -89,6 +90,7 @@ noinst_HEADERS =                             \
     guac_rdpdr/rdpdr_fs_service.h            \
     guac_rdpdr/rdpdr_messages.h              \
     guac_rdpdr/rdpdr_printer.h               \
+    guac_rdpdr/rdpdr_print_job.h             \
     guac_rdpdr/rdpdr_service.h               \
     guac_rdpsnd/rdpsnd_messages.h            \
     guac_rdpsnd/rdpsnd_service.h             \

http://git-wip-us.apache.org/repos/asf/incubator-guacamole-server/blob/3fc43fba/src/protocols/rdp/guac_rdpdr/rdpdr_print_job.c
----------------------------------------------------------------------
diff --git a/src/protocols/rdp/guac_rdpdr/rdpdr_print_job.c 
b/src/protocols/rdp/guac_rdpdr/rdpdr_print_job.c
new file mode 100644
index 0000000..8beaac7
--- /dev/null
+++ b/src/protocols/rdp/guac_rdpdr/rdpdr_print_job.c
@@ -0,0 +1,581 @@
+/*
+ * 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.
+ */
+
+
+#include "config.h"
+#include "rdpdr_print_job.h"
+
+#include <guacamole/client.h>
+#include <guacamole/protocol.h>
+#include <guacamole/socket.h>
+#include <guacamole/stream.h>
+#include <guacamole/user.h>
+
+#include <errno.h>
+#include <pthread.h>
+#include <stdlib.h>
+#include <string.h>
+#include <unistd.h>
+
+/**
+ * The command to run when filtering postscript to produce PDF. This must be
+ * a NULL-terminated array of arguments, where the first argument is the name
+ * of the file to run.
+ */
+char* const guac_rdpdr_pdf_filter_command[] = {
+    "gs",
+    "-q",
+    "-dNOPAUSE",
+    "-dBATCH",
+    "-dSAFER",
+    "-dPARANOIDSAFER",
+    "-sDEVICE=pdfwrite",
+    "-sOutputFile=-",
+    "-c",
+    ".setpdfwrite",
+    "-sstdout=/dev/null",
+    "-f",
+    "-",
+    NULL
+};
+
+/**
+ * Updates the state of the given print job. Any threads currently blocked by a
+ * call to guac_rdpdr_print_job_wait_for_ack() will be unblocked.
+ *
+ * @param job
+ *     The print job whose state should be updated.
+ *
+ * @param state
+ *     The new state to assign to the given print job.
+ */
+static void guac_rdpdr_print_job_set_state(guac_rdpdr_print_job* job,
+        guac_rdpdr_print_job_state state) {
+
+    pthread_mutex_lock(&(job->state_lock));
+
+    /* Update stream state, signalling modification */
+    job->state = state;
+    pthread_cond_signal(&(job->state_modified));
+
+    pthread_mutex_unlock(&(job->state_lock));
+
+}
+
+/**
+ * Suspends execution of the current thread until the state of the given print
+ * job is not GUAC_RDPDR_PRINT_JOB_WAITING_FOR_ACK. If the state of the print
+ * job is GUAC_RDPDR_PRINT_JOB_ACK_RECEIVED, the state is automatically reset
+ * back to GUAC_RDPDR_PRINT_JOB_WAITING_FOR_ACK prior to returning.
+ *
+ * @param job
+ *     The print job to wait for.
+ *
+ * @return
+ *     Zero if the state of the print job is GUAC_RDPDR_PRINT_JOB_CLOSED,
+ *     non-zero if the state was GUAC_RDPDR_PRINT_JOB_ACK_RECEIVED and has been
+ *     automatically reset to GUAC_RDPDR_PRINT_JOB_WAITING_FOR_ACK.
+ */
+static int guac_rdpdr_print_job_wait_for_ack(guac_rdpdr_print_job* job) {
+
+    /* Wait for ack if stream open and not yet received */
+    pthread_mutex_lock(&(job->state_lock));
+    if (job->state == GUAC_RDPDR_PRINT_JOB_WAITING_FOR_ACK)
+        pthread_cond_wait(&job->state_modified, &job->state_lock);
+
+    /* Reset state if ack received */
+    int got_ack = (job->state == GUAC_RDPDR_PRINT_JOB_ACK_RECEIVED);
+    if (got_ack)
+        job->state = GUAC_RDPDR_PRINT_JOB_WAITING_FOR_ACK;
+
+    /* Return whether ack was successfully received */
+    pthread_mutex_unlock(&(job->state_lock));
+    return got_ack;
+
+}
+
+/**
+ * Sends a "file" instruction to the given user describing the PDF file that
+ * will be sent using the output of the given print job. If the given user no
+ * longer exists, the print stream will be automatically terminated.
+ *
+ * @param user
+ *     The user receiving the "file" instruction.
+ *
+ * @param data
+ *     A pointer to the guac_rdpdr_print_job representing the print job being
+ *     streamed.
+ *
+ * @return
+ *     Always NULL.
+ */
+static void* guac_rdpdr_print_job_begin_stream(guac_user* user, void* data) {
+
+    guac_rdpdr_print_job* job = (guac_rdpdr_print_job*) data;
+    guac_client_log(job->client, GUAC_LOG_DEBUG, "Beginning print stream: %s",
+            job->filename);
+
+    /* Kill job and do nothing if user no longer exists */
+    if (user == NULL) {
+        guac_rdpdr_print_job_kill(job);
+        return NULL;
+    }
+
+    /* Send document as a PDF file stream */
+    guac_protocol_send_file(user->socket, job->stream,
+            "application/pdf", job->filename);
+
+    guac_socket_flush(user->socket);
+    return NULL;
+
+}
+
+/**
+ * Sends a "blob" instruction to the given user containing the provided data
+ * along the stream associated with the provided print job. If the given user
+ * no longer exists, the print stream will be automatically terminated.
+ *
+ * @param user
+ *     The user receiving the "blob" instruction.
+ *
+ * @param data
+ *     A pointer to an guac_rdpdr_print_blob structure containing the data to
+ *     be written, the number of bytes being written, and the print job being
+ *     streamed.
+ *
+ * @return
+ *     Always NULL.
+ */
+static void* guac_rdpdr_print_job_send_blob(guac_user* user, void* data) {
+
+    guac_rdpdr_print_blob* blob = (guac_rdpdr_print_blob*) data;
+    guac_rdpdr_print_job* job = blob->job;
+
+    guac_client_log(job->client, GUAC_LOG_DEBUG, "Sending %i byte(s) "
+            "of filtered output.", blob->length);
+
+    /* Kill job and do nothing if user no longer exists */
+    if (user == NULL) {
+        guac_rdpdr_print_job_kill(job);
+        return NULL;
+    }
+
+    /* Send single blob of print data */
+    guac_protocol_send_blob(user->socket, job->stream,
+            blob->buffer, blob->length);
+
+    guac_socket_flush(user->socket);
+    return NULL;
+
+}
+
+/**
+ * Sends an "end" instruction to the given user, closing the stream associated
+ * with the given print job. If the given user no longer exists, the print
+ * stream will be automatically terminated.
+ *
+ * @param user
+ *     The user receiving the "end" instruction.
+ *
+ * @param data
+ *     A pointer to the guac_rdpdr_print_job representing the print job being
+ *     streamed.
+ *
+ * @return
+ *     Always NULL.
+ */
+static void* guac_rdpdr_print_job_end_stream(guac_user* user, void* data) {
+
+    guac_rdpdr_print_job* job = (guac_rdpdr_print_job*) data;
+    guac_client_log(job->client, GUAC_LOG_DEBUG, "End of print stream.");
+
+    /* Kill job and do nothing if user no longer exists */
+    if (user == NULL) {
+        guac_rdpdr_print_job_kill(job);
+        return NULL;
+    }
+
+    /* Explicitly close down stream */
+    guac_protocol_send_end(user->socket, job->stream);
+    guac_socket_flush(user->socket);
+
+    /* Clean up our end of the stream */
+    guac_user_free_stream(job->user, job->stream);
+
+    return NULL;
+
+}
+
+/**
+ * Handler for "ack" messages received in response to printed data. Additional
+ * data will be sent as a result or, if no data remains, the stream will be
+ * terminated. It is required that the data pointer of the provided stream be
+ * set to the file descriptor from which the printed data should be read.
+ *
+ * @param user
+ *     The user to whom the printed data is being sent.
+ *
+ * @param stream
+ *     The stream along which the printed data is to be sent. The data pointer
+ *     of this stream MUST be set to the file descriptor from which the data
+ *     being sent is to be read.
+ *
+ * @param message
+ *     An arbitrary, human-readable message describing the success/failure of
+ *     the operation being acknowledged (either stream creation or receipt of
+ *     a blob).
+ *
+ * @param status
+ *     The status code describing the success/failure of the operation being
+ *     acknowledged (either stream creation or receipt of a blob).
+ *
+ * @return
+ *     Always zero.
+ */
+static int guac_rdpdr_print_filter_ack_handler(guac_user* user,
+        guac_stream* stream, char* message, guac_protocol_status status) {
+
+    guac_rdpdr_print_job* job = (guac_rdpdr_print_job*) stream->data;
+
+    /* Update state for successful acks */
+    if (status == GUAC_PROTOCOL_STATUS_SUCCESS)
+        guac_rdpdr_print_job_set_state(job, GUAC_RDPDR_PRINT_JOB_ACK_RECEIVED);
+
+    /* Terminate stream if ack signals an error */
+    else {
+
+        /* Note that the stream was aborted by the user */
+        guac_client_log(job->client, GUAC_LOG_INFO, "User explicitly aborted "
+                "print stream.");
+
+        /* Kill job (the results will no longer be received) */
+        guac_rdpdr_print_job_kill(job);
+
+    }
+
+    return 0;
+
+}
+
+/**
+ * Forks a new print filtering process which accepts PostScript input and
+ * produces PDF output. File descriptors for writing input and reading output
+ * will automatically be allocated and must be manually closed when processing
+ * is complete.
+ *
+ * @param client
+ *     The guac_client associated with the print job for which this filter
+ *     process is being created.
+ *
+ * @param input_fd
+ *     A pointer to an int which should receive the input file descriptor of
+ *     the filter process. PostScript input for the filter process should be
+ *     written to this file descriptor.
+ *
+ * @param output_fd
+ *     A pointer to an int which should receive the output file descriptor of
+ *     the filter process. PDF output from the filter process must be
+ *     continuously read from this file descriptor or the pipeline may block.
+ *
+ * @return
+ *     The PID of the filter process, or -1 if the filter process could not be
+ *     created. If the filter process could not be created, the values assigned
+ *     through input_fd and output_fd are undefined.
+ */
+static pid_t guac_rdpdr_create_filter_process(guac_client* client,
+        int* input_fd, int* output_fd) {
+
+    int child_pid;
+    int stdin_pipe[2];
+    int stdout_pipe[2];
+
+    /* Create STDIN pipe */
+    if (pipe(stdin_pipe)) {
+        guac_client_log(client, GUAC_LOG_ERROR, "Unable to create STDIN "
+                "pipe for PDF filter process: %s", strerror(errno));
+        return -1;
+    }
+
+    /* Create STDOUT pipe */
+    if (pipe(stdout_pipe)) {
+        guac_client_log(client, GUAC_LOG_ERROR, "Unable to create STDOUT "
+                "pipe for PDF filter process: %s", strerror(errno));
+        close(stdin_pipe[0]);
+        close(stdin_pipe[1]);
+        return -1;
+    }
+
+    /* Store parent side of stdin/stdout */
+    *input_fd = stdin_pipe[1];
+    *output_fd = stdout_pipe[0];
+
+    /* Fork child process */
+    child_pid = fork();
+
+    /* Log fork errors */
+    if (child_pid == -1) {
+        guac_client_log(client, GUAC_LOG_ERROR, "Unable to fork PDF filter "
+                "process: %s", strerror(errno));
+        close(stdin_pipe[0]);
+        close(stdin_pipe[1]);
+        close(stdout_pipe[0]);
+        close(stdout_pipe[1]);
+        return -1;
+    }
+
+    /* Child process */
+    if (child_pid == 0) {
+
+        /* Close unneeded ends of pipe */
+        close(stdin_pipe[1]);
+        close(stdout_pipe[0]);
+
+        /* Reassign file descriptors as STDIN/STDOUT */
+        dup2(stdin_pipe[0], STDIN_FILENO);
+        dup2(stdout_pipe[1], STDOUT_FILENO);
+
+        /* Run PDF filter */
+        guac_client_log(client, GUAC_LOG_INFO, "Running %s",
+                guac_rdpdr_pdf_filter_command[0]);
+        if (execvp(guac_rdpdr_pdf_filter_command[0],
+                    guac_rdpdr_pdf_filter_command) < 0)
+            guac_client_log(client, GUAC_LOG_ERROR, "Unable to execute PDF "
+                    "filter command: %s", strerror(errno));
+        else
+            guac_client_log(client, GUAC_LOG_ERROR, "Unable to execute PDF "
+                    "filter command, but no error given");
+
+        /* Terminate child process */
+        exit(1);
+
+    }
+
+    /* Log fork success */
+    guac_client_log(client, GUAC_LOG_INFO, "Created PDF filter process "
+            "PID=%i", child_pid);
+
+    /* Close unneeded ends of pipe */
+    close(stdin_pipe[0]);
+    close(stdout_pipe[1]);
+    return child_pid;
+
+}
+
+/**
+ * Thread which continuously reads from the output file descriptor associated
+ * with the given print job, writing filtered PDF output to the associated
+ * Guacamole stream, and terminating only after the print job has completed
+ * processing or the associated Guacamole stream has closed.
+ *
+ * @param data
+ *     A pointer to the guac_rdpdr_print_job representing the print job that
+ *     should be read.
+ *
+ * @return
+ *     Always NULL.
+ */
+static void* guac_rdpdr_print_job_output_thread(void* data) {
+
+    int length;
+    char buffer[6048];
+
+    guac_rdpdr_print_job* job = (guac_rdpdr_print_job*) data;
+    guac_client_log(job->client, GUAC_LOG_DEBUG, "Reading output from filter "
+            "process...");
+
+    /* Read continuously while data remains */
+    while ((length = read(job->output_fd, buffer, sizeof(buffer))) > 0) {
+
+        /* Wait for client to be ready for blob */
+        if (guac_rdpdr_print_job_wait_for_ack(job)) {
+
+            guac_rdpdr_print_blob blob = {
+                .job    = job,
+                .buffer = buffer,
+                .length = length
+            };
+
+            /* Write a single blob of output */
+            guac_client_for_user(job->client, job->user,
+                    guac_rdpdr_print_job_send_blob, &blob);
+
+        }
+
+        /* Abort if stream is closed */
+        else {
+            guac_client_log(job->client, GUAC_LOG_DEBUG, "Print stream "
+                    "explicitly aborted.");
+            break;
+        }
+
+    }
+
+    /* Warn of read errors */
+    if (length < 0)
+        guac_client_log(job->client, GUAC_LOG_ERROR,
+                "Error reading from filter: %s", strerror(errno));
+
+    /* Terminate stream */
+    guac_client_for_user(job->client, job->user,
+            guac_rdpdr_print_job_end_stream, job);
+
+    /* Ensure all associated file descriptors are closed */
+    close(job->input_fd);
+    close(job->output_fd);
+
+    guac_client_log(job->client, GUAC_LOG_DEBUG, "Print job completed.");
+    return NULL;
+
+}
+
+void* guac_rdpdr_print_job_alloc(guac_user* user, void* data) {
+
+    /* Allocate nothing if user does not exist */
+    if (user == NULL)
+        return NULL;
+
+    /* Allocate stream for print job output */
+    guac_stream* stream = guac_user_alloc_stream(user);
+    if (stream == NULL)
+        return NULL;
+
+    /* Bail early if allocation fails */
+    guac_rdpdr_print_job* job = malloc(sizeof(guac_rdpdr_print_job));
+    if (job == NULL)
+        return NULL;
+
+    /* Associate job with stream and dependent data */
+    job->client = user->client;
+    job->user = user;
+    job->stream = stream;
+    job->bytes_received = 0;
+
+    /* Set default filename for job */
+    strcpy(job->filename, GUAC_RDPDR_PRINT_JOB_DEFAULT_FILENAME);
+
+    /* Prepare stream for receipt of acks */
+    stream->ack_handler = guac_rdpdr_print_filter_ack_handler;
+    stream->data = job;
+
+    /* Create print filter process */
+    job->filter_pid = guac_rdpdr_create_filter_process(job->client,
+            &job->input_fd, &job->output_fd);
+
+    /* Abort if print filter process cannot be created */
+    if (job->filter_pid == -1) {
+        guac_user_free_stream(user, stream);
+        free(job);
+        return NULL;
+    }
+
+    /* Init stream state signal and lock */
+    job->state = GUAC_RDPDR_PRINT_JOB_WAITING_FOR_ACK;
+    pthread_cond_init(&job->state_modified, NULL);
+    pthread_mutex_init(&job->state_lock, NULL);
+
+    /* Start output thread */
+    pthread_create(&job->output_thread, NULL,
+            guac_rdpdr_print_job_output_thread, job);
+
+    /* Print job allocated successfully */
+    return job;
+
+}
+
+int guac_rdpdr_print_job_write(guac_rdpdr_print_job* job,
+        void* buffer, int length) {
+
+    /* Create print job, if not yet created */
+    if (job->bytes_received == 0) {
+
+        char* filename = job->filename;
+        unsigned char* search = buffer;
+        int i;
+
+        /* Search for filename within buffer */
+        for (i=0; i<length-9 && i < 2048; i++) {
+
+            /* If title. use as filename */
+            if (memcmp(search, "%%Title: ", 9) == 0) {
+
+                /* Skip past "%%Title: " */
+                search += 9;
+
+                /* Copy as much of title as reasonable */
+                int j;
+                for (j=0; j < GUAC_RDPDR_PRINT_JOB_FILENAME_MAX_LENGTH - 5 /* 
extension + 1 */ && i<length; i++, j++) {
+
+                    /* Get character, stop at EOL */
+                    char c = *(search++);
+                    if (c == '\r' || c == '\n')
+                        break;
+
+                    /* Copy to filename */
+                    filename[j] = c;
+
+                }
+
+                /* Append filename with extension */
+                strcpy(&(filename[j]), ".pdf");
+                break;
+            }
+
+            /* Next character */
+            search++;
+
+        }
+
+        /* Begin print stream */
+        guac_client_for_user(job->client, job->user,
+                guac_rdpdr_print_job_begin_stream, job);
+
+    } /* end if print job beginning */
+
+    /* Update counter of bytes received */
+    job->bytes_received += length;
+
+    /* Write data to filter process */
+    return write(job->input_fd, buffer, length);
+
+}
+
+void guac_rdpdr_print_job_free(guac_rdpdr_print_job* job) {
+
+    /* No more input will be provided */
+    close(job->input_fd);
+
+    /* Wait for job to terminate */
+    pthread_join(job->output_thread, NULL);
+
+    /* Free base structure */
+    free(job);
+
+}
+
+void guac_rdpdr_print_job_kill(guac_rdpdr_print_job* job) {
+
+    /* Stop all handling of I/O */
+    close(job->input_fd);
+    close(job->output_fd);
+
+    /* Mark stream as closed */
+    guac_rdpdr_print_job_set_state(job, GUAC_RDPDR_PRINT_JOB_CLOSED);
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-guacamole-server/blob/3fc43fba/src/protocols/rdp/guac_rdpdr/rdpdr_print_job.h
----------------------------------------------------------------------
diff --git a/src/protocols/rdp/guac_rdpdr/rdpdr_print_job.h 
b/src/protocols/rdp/guac_rdpdr/rdpdr_print_job.h
new file mode 100644
index 0000000..81b1863
--- /dev/null
+++ b/src/protocols/rdp/guac_rdpdr/rdpdr_print_job.h
@@ -0,0 +1,226 @@
+/*
+ * 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.
+ */
+
+#ifndef GUAC_RDPDR_PRINT_JOB_H
+#define GUAC_RDPDR_PRINT_JOB_H
+
+#include "config.h"
+
+#include <guacamole/client.h>
+#include <guacamole/stream.h>
+#include <guacamole/user.h>
+
+#include <pthread.h>
+#include <unistd.h>
+
+/**
+ * The maximum number of bytes in the filename of an RDPDR print job sent as a
+ * file over the Guacamole protocol, including NULL terminator.
+ */
+#define GUAC_RDPDR_PRINT_JOB_FILENAME_MAX_LENGTH 1024
+
+/**
+ * The default filename to use for the PDF output of an RDPDR print job if no
+ * document title can be found within the printed data.
+ */
+#define GUAC_RDPDR_PRINT_JOB_DEFAULT_FILENAME "guacamole-print.pdf"
+
+/**
+ * The current state of an RDPDR print job.
+ */
+typedef enum guac_rdpdr_print_job_state {
+
+    /**
+     * The print stream has been opened with the Guacamole client, but the
+     * client has not yet confirmed that it is ready to receive data.
+     */
+    GUAC_RDPDR_PRINT_JOB_WAITING_FOR_ACK,
+
+    /**
+     * The print stream has been opened with the Guacamole client, and the
+     * client has responded with an "ack", confirming that it is ready to
+     * receive data (or that data has been received and it is ready to receive
+     * more).
+     */
+    GUAC_RDPDR_PRINT_JOB_ACK_RECEIVED,
+
+    /**
+     * The print stream has been closed or the printer is terminating, and no
+     * further data should be sent to the client.
+     */
+    GUAC_RDPDR_PRINT_JOB_CLOSED
+
+} guac_rdpdr_print_job_state;
+
+/**
+ * Data specific to an instance of the printer device.
+ */
+typedef struct guac_rdpdr_print_job {
+
+    guac_client* client;
+
+    /**
+     * The user receiving the output from the print job.
+     */
+    guac_user* user;
+
+    /**
+     * The stream along which the print job output should be sent.
+     */
+    guac_stream* stream;
+
+    /**
+     * The PID of the print filter process converting PostScript data into PDF.
+     */
+    pid_t filter_pid;
+
+    /**
+     * The filename that should be used when the converted PDF output is
+     * streamed to the Guacamole user. This value will be automatically
+     * determined based on the contents of the printed document.
+     */
+    char filename[GUAC_RDPDR_PRINT_JOB_FILENAME_MAX_LENGTH];
+
+    /**
+     * File descriptor that should be written to when sending documents to the
+     * printer.
+     */
+    int input_fd;
+
+    /**
+     * File descriptor that should be read from when receiving output from the
+     * printer.
+     */
+    int output_fd;
+
+    /**
+     * The current state of the print stream, dependent on whether the client
+     * has acknowledged creation of the stream, whether the client has
+     * acknowledged receipt of data along the steam, and whether the print
+     * stream itself has closed.
+     */
+    guac_rdpdr_print_job_state state;
+
+    /**
+     * Lock which is acquired prior to modifying the state property or waiting
+     * on the state_modified conditional.
+     */
+    pthread_mutex_t state_lock;
+
+    /**
+     * Conditional which signals modification to the state property of this
+     * structure.
+     */
+    pthread_cond_t state_modified;
+
+    /**
+     * Thread which transfers data from the printer to the Guacamole client.
+     */
+    pthread_t output_thread;
+
+    /**
+     * The number of bytes received in the current print job.
+     */
+    int bytes_received;
+
+} guac_rdpdr_print_job;
+
+/**
+ * A blob of print data being sent to the Guacamole user.
+ */
+typedef struct guac_rdpdr_print_blob {
+
+    /**
+     * The print job which generated the data being sent.
+     */
+    guac_rdpdr_print_job* job;
+
+    /**
+     * The data being sent.
+     */
+    void* buffer;
+
+    /**
+     * The number of bytes of data being sent.
+     */
+    int length;
+
+} guac_rdpdr_print_blob;
+
+/**
+ * Allocates a new print job for the given user. It is expected that this
+ * function will be invoked via a call to guac_client_for_user() or
+ * guac_client_for_owner().
+ *
+ * @param user
+ *     The user that should receive the output from the print job.
+ *
+ * @param data
+ *     An arbitrary data parameter required by guac_client_for_user() and
+ *     guac_client_for_owner() but ignored by this function. This should
+ *     always be NULL.
+ *
+ * @return
+ *     A pointer to a newly-allocated guac_rdpdr_print_job, or NULL if the
+ *     print job could not be created.
+ */
+void* guac_rdpdr_print_job_alloc(guac_user* user, void* data);
+
+/**
+ * Writes PostScript print data to the given active print job. The print job
+ * will automatically convert this data to PDF, streaming the result to the
+ * Guacamole user associated with the print job. This function may block if
+ * the print job is not yet ready for more data.
+ *
+ * @param buffer
+ *     The PostScript print data to write to the given print job.
+ *
+ * @param length
+ *     The number of bytes of PostScript print data to write.
+ *
+ * @return
+ *     The number of bytes written, or -1 if an error occurs which prevents
+ *     further writes.
+ */
+int guac_rdpdr_print_job_write(guac_rdpdr_print_job* job,
+        void* buffer, int length);
+
+/**
+ * Frees the memory associated with the given print job, closing all underlying
+ * file descriptors, and ending the file transfer to the associated Guacamole
+ * user. This function may block if the print filter process has not yet
+ * finished processing the received data.
+ *
+ * @param job
+ *     The print job to free.
+ */
+void guac_rdpdr_print_job_free(guac_rdpdr_print_job* job);
+
+/**
+ * Forcibly kills the given print job, stopping all associated processing and
+ * streaming. The memory associated with the print job will still need to be
+ * reclaimed via guac_rdpdr_print_job_free().
+ *
+ * @param job
+ *     The print job to kill.
+ */
+void guac_rdpdr_print_job_kill(guac_rdpdr_print_job* job);
+
+#endif
+

http://git-wip-us.apache.org/repos/asf/incubator-guacamole-server/blob/3fc43fba/src/protocols/rdp/guac_rdpdr/rdpdr_printer.c
----------------------------------------------------------------------
diff --git a/src/protocols/rdp/guac_rdpdr/rdpdr_printer.c 
b/src/protocols/rdp/guac_rdpdr/rdpdr_printer.c
index 41c3663..cd7c517 100644
--- a/src/protocols/rdp/guac_rdpdr/rdpdr_printer.c
+++ b/src/protocols/rdp/guac_rdpdr/rdpdr_printer.c
@@ -21,6 +21,7 @@
 
 #include "rdpdr_messages.h"
 #include "rdpdr_printer.h"
+#include "rdpdr_print_job.h"
 #include "rdpdr_service.h"
 #include "rdp_status.h"
 
@@ -38,330 +39,60 @@
 #endif
 
 #include <errno.h>
+#include <pthread.h>
 #include <stdint.h>
 #include <stdlib.h>
 #include <string.h>
 #include <unistd.h>
 
-/* Command to run GhostScript safely as a filter writing PDF */
-char* const guac_rdpdr_pdf_filter_command[] = {
-    "gs",
-    "-q",
-    "-dNOPAUSE",
-    "-dBATCH",
-    "-dSAFER",
-    "-dPARANOIDSAFER",
-    "-sDEVICE=pdfwrite",
-    "-sOutputFile=-",
-    "-c",
-    ".setpdfwrite",
-    "-sstdout=/dev/null",
-    "-f",
-    "-",
-    NULL
-};
-
-/**
- * Handler for "ack" messages received in response to printed data. Additional
- * data will be sent as a result or, if no data remains, the stream will be
- * terminated. It is required that the data pointer of the provided stream be
- * set to the file descriptor from which the printed data should be read.
- *
- * @param user
- *     The user to whom the printed data is being sent.
- *
- * @param stream
- *     The stream along which the printed data is to be sent. The data pointer
- *     of this stream MUST be set to the file descriptor from which the data
- *     being sent is to be read.
- *
- * @param message
- *     An arbitrary, human-readable message describing the success/failure of
- *     the operation being acknowledged (either stream creation or receipt of
- *     a blob).
- *
- * @param status
- *     The status code describing the success/failure of the operation being
- *     acknowledged (either stream creation or receipt of a blob).
- *
- * @return
- *     Always zero.
- */
-static int guac_rdpdr_print_filter_ack_handler(guac_user* user,
-        guac_stream* stream, char* message, guac_protocol_status status) {
-
-    char buffer[6048];
-
-    /* Pull file descriptor from stream data */
-    int fd = (intptr_t) stream->data;
-
-    /* Reading only if ack reports success */
-    if (status == GUAC_PROTOCOL_STATUS_SUCCESS) {
-
-        /* Write a single blob of output */
-        int length = read(fd, buffer, sizeof(buffer));
-        if (length > 0) {
-            guac_protocol_send_blob(user->socket, stream, buffer, length);
-            guac_socket_flush(user->socket);
-            return 0;
-        }
-
-        /* Warn of read errors, fall through to termination */
-        else if (length < 0)
-            guac_user_log(user, GUAC_LOG_ERROR,
-                    "Error reading from filter: %s", strerror(errno));
-
-    }
-
-    /* Note if stream aborted by user, fall through to termination */
-    else
-        guac_user_log(user, GUAC_LOG_INFO, "Print stream aborted.");
-
-    /* Explicitly close down stream */
-    guac_protocol_send_end(user->socket, stream);
-    guac_socket_flush(user->socket);
-
-    /* Clean up our end of the stream */
-    guac_user_free_stream(user, stream);
-    close(fd);
-
-    return 0;
-
-}
-
-static int guac_rdpdr_create_print_process(guac_rdpdr_device* device) {
-
-    guac_rdpdr_printer_data* printer_data = (guac_rdpdr_printer_data*) 
device->data;
-
-    int child_pid;
-    int stdin_pipe[2];
-    int stdout_pipe[2];
-
-    /* Create STDIN pipe */
-    if (pipe(stdin_pipe)) {
-        guac_client_log(device->rdpdr->client, GUAC_LOG_ERROR,
-                "Unable to create STDIN pipe for PDF filter process: %s", 
strerror(errno));
-        return 1;
-    }
-
-    /* Create STDOUT pipe */
-    if (pipe(stdout_pipe)) {
-        guac_client_log(device->rdpdr->client, GUAC_LOG_ERROR,
-                "Unable to create STDIN pipe for PDF filter process: %s", 
strerror(errno));
-        close(stdin_pipe[0]);
-        close(stdin_pipe[1]);
-        return 1;
-    }
-
-    /* Store our side of stdin/stdout */
-    printer_data->printer_input  = stdin_pipe[1];
-    printer_data->printer_output = stdout_pipe[0];
-
-    /* Fork child process */
-    child_pid = fork();
-
-    /* Log fork errors */
-    if (child_pid == -1) {
-        guac_client_log(device->rdpdr->client, GUAC_LOG_ERROR,
-                "Unable to fork PDF filter process: %s", strerror(errno));
-        close(stdin_pipe[0]);
-        close(stdin_pipe[1]);
-        close(stdout_pipe[0]);
-        close(stdout_pipe[1]);
-        return 1;
-    }
-
-    /* Child process */
-    if (child_pid == 0) {
-
-        /* Close unneeded ends of pipe */
-        close(stdin_pipe[1]);
-        close(stdout_pipe[0]);
-
-        /* Reassign file descriptors as STDIN/STDOUT */
-        dup2(stdin_pipe[0], STDIN_FILENO);
-        dup2(stdout_pipe[1], STDOUT_FILENO);
-
-        /* Run PDF filter */
-        guac_client_log(device->rdpdr->client, GUAC_LOG_INFO, "Running %s", 
guac_rdpdr_pdf_filter_command[0]);
-        if (execvp(guac_rdpdr_pdf_filter_command[0], 
guac_rdpdr_pdf_filter_command) < 0)
-            guac_client_log(device->rdpdr->client, GUAC_LOG_ERROR, "Unable to 
execute PDF filter command: %s", strerror(errno));
-        else
-            guac_client_log(device->rdpdr->client, GUAC_LOG_ERROR, "Unable to 
execute PDF filter command, but no error given");
-
-        /* Terminate child process */
-        exit(1);
-
-    }
-
-    /* Log fork success */
-    guac_client_log(device->rdpdr->client, GUAC_LOG_INFO, "Created PDF filter 
process PID=%i", child_pid);
-
-    /* Close unneeded ends of pipe */
-    close(stdin_pipe[0]);
-    close(stdout_pipe[1]);
-    return 0;
-
-}
-
 void guac_rdpdr_process_print_job_create(guac_rdpdr_device* device,
         wStream* input_stream, int completion_id) {
 
-    guac_rdpdr_printer_data* printer_data =
-        (guac_rdpdr_printer_data*) device->data;
+    /* Log creation of print job */
+    guac_client_log(device->rdpdr->client, GUAC_LOG_INFO, "Print job created");
 
+    /* Create print job */
+    device->data = guac_client_for_owner(device->rdpdr->client,
+            guac_rdpdr_print_job_alloc, NULL);
+
+    /* Respond with success */
     wStream* output_stream = guac_rdpdr_new_io_completion(device,
             completion_id, STATUS_SUCCESS, 4);
 
-    /* No bytes received yet */
-    printer_data->bytes_received = 0;
     Stream_Write_UINT32(output_stream, 0); /* fileId */
-
     svc_plugin_send((rdpSvcPlugin*) device->rdpdr, output_stream);
 
 }
 
-/**
- * Given data representing a print device with a pending pring job, allocates a
- * new stream for the given user, associating it with the provided data and
- * returning the resulting guac_stream. The stream will be pre-configured to
- * send blobs of print data in response to "ack" messages received from the
- * user. If the given user is NULL, no stream will be allocated, and the
- * print job will be immediately aborted.
- *
- * @param user
- *     The user to whom the print job is being sent, or NULL if no stream
- *     should be allocated.
- *
- * @param data
- *     A pointer to the guac_rdpdr_device instance associated with the new
- *     print job.
- *
- * @return
- *     The guac_stream allocated for the new print job, or NULL if no stream
- *     could be allocated.
- */
-static void* guac_rdpdr_alloc_printer_stream(guac_user* owner, void* data) {
-
-    guac_rdpdr_device* device = (guac_rdpdr_device*) data;
-    guac_rdpdr_printer_data* printer_data =
-        (guac_rdpdr_printer_data*) device->data;
-
-    /* Abort immediately if there is no owner */
-    if (owner == NULL) {
-        close(printer_data->printer_output);
-        close(printer_data->printer_input);
-        printer_data->printer_output = -1;
-        printer_data->printer_input = -1;
-        return NULL;
-    }
-
-    /* Allocate stream for owner */
-    guac_stream* stream = guac_user_alloc_stream(owner);
-    stream->ack_handler = guac_rdpdr_print_filter_ack_handler;
-    stream->data = (void*) (intptr_t) printer_data->printer_output;
-
-    return stream;
-
-}
-
 void guac_rdpdr_process_print_job_write(guac_rdpdr_device* device,
         wStream* input_stream, int completion_id) {
 
-    guac_rdpdr_printer_data* printer_data = (guac_rdpdr_printer_data*) 
device->data;
-    int status=0, length;
-    unsigned char* buffer;
+    guac_rdpdr_print_job* job = (guac_rdpdr_print_job*) device->data;
 
-    wStream* output_stream;
+    unsigned char* buffer;
+    int length;
+    int status;
 
+    /* Read buffer of print data */
     Stream_Read_UINT32(input_stream, length);
     Stream_Seek(input_stream, 8);  /* Offset */
     Stream_Seek(input_stream, 20); /* Padding */
     buffer = Stream_Pointer(input_stream);
 
-    /* Create print job, if not yet created */
-    if (printer_data->bytes_received == 0) {
-
-        char filename[1024] = "guacamole-print.pdf";
-        unsigned char* search = buffer;
-        int i;
-
-        /* Search for filename within buffer */
-        for (i=0; i<length-9 && i < 2048; i++) {
-
-            /* If title. use as filename */
-            if (memcmp(search, "%%Title: ", 9) == 0) {
-
-                /* Skip past "%%Title: " */
-                search += 9;
-
-                /* Copy as much of title as reasonable */
-                int j;
-                for (j=0; j<sizeof(filename) - 5 /* extension + 1 */ && 
i<length; i++, j++) {
-
-                    /* Get character, stop at EOL */
-                    char c = *(search++);
-                    if (c == '\r' || c == '\n')
-                        break;
-
-                    /* Copy to filename */
-                    filename[j] = c;
-
-                }
-
-                /* Append filename with extension */
-                strcpy(&(filename[j]), ".pdf");
-                break;
-            }
-
-            /* Next character */
-            search++;
-
-        }
-
-        /* Start print process */
-        if (guac_rdpdr_create_print_process(device) != 0) {
-            status = STATUS_DEVICE_OFF_LINE;
-            length = 0;
-        }
-
-        /* If print started successfully, create outbound stream */
-        else {
-
-            guac_client_log(device->rdpdr->client, GUAC_LOG_INFO,
-                    "Print job created");
-
-            /* Allocate stream */
-            guac_stream* stream = (guac_stream*) guac_client_for_owner(
-                    device->rdpdr->client, guac_rdpdr_alloc_printer_stream,
-                    device);
-
-            /* Begin file if stream allocation was successful */
-            if (stream != NULL)
-                guac_protocol_send_file(device->rdpdr->client->socket, stream,
-                        "application/pdf", filename);
-
-        }
-
-    } /* end if print job beginning */
-
-    printer_data->bytes_received += length;
-
-    /* If not yet failed, write received data */
-    if (status == 0) {
-
-        /* Write data to printer, translate output for RDP */
-        length = write(printer_data->printer_input, buffer, length);
-        if (length == -1) {
-            guac_client_log(device->rdpdr->client, GUAC_LOG_ERROR, "Error 
writing to printer: %s", strerror(errno));
-            status = STATUS_DEVICE_OFF_LINE;
-            length = 0;
-        }
+    /* Write data only if job exists, translating status for RDP */
+    if (job != NULL && (length = guac_rdpdr_print_job_write(job,
+                    buffer, length)) >= 0) {
+        status = STATUS_SUCCESS;
+    }
 
+    /* Report device offline if write fails */
+    else {
+        status = STATUS_DEVICE_OFF_LINE;
+        length = 0;
     }
 
-    output_stream = guac_rdpdr_new_io_completion(device, completion_id,
-            status, 5);
+    wStream* output_stream = guac_rdpdr_new_io_completion(device,
+            completion_id, status, 5);
 
     Stream_Write_UINT32(output_stream, length);
     Stream_Write_UINT8(output_stream, 0); /* Padding */
@@ -373,20 +104,21 @@ void 
guac_rdpdr_process_print_job_write(guac_rdpdr_device* device,
 void guac_rdpdr_process_print_job_close(guac_rdpdr_device* device,
         wStream* input_stream, int completion_id) {
 
-    guac_rdpdr_printer_data* printer_data =
-        (guac_rdpdr_printer_data*) device->data;
+    /* End print job */
+    guac_rdpdr_print_job* job = (guac_rdpdr_print_job*) device->data;
+    if (job != NULL) {
+        guac_rdpdr_print_job_free(job);
+        device->data = NULL;
+    }
 
     wStream* output_stream = guac_rdpdr_new_io_completion(device,
             completion_id, STATUS_SUCCESS, 4);
 
     Stream_Write_UINT32(output_stream, 0); /* Padding */
+    svc_plugin_send((rdpSvcPlugin*) device->rdpdr, output_stream);
 
-    /* Close input - the Guacamole stream will continue while output remains */
-    close(printer_data->printer_input);
-    printer_data->printer_input = -1;
-
+    /* Log end of print job */
     guac_client_log(device->rdpdr->client, GUAC_LOG_INFO, "Print job closed");
-    svc_plugin_send((rdpSvcPlugin*) device->rdpdr, output_stream);
 
 }
 
@@ -447,19 +179,10 @@ static void 
guac_rdpdr_device_printer_iorequest_handler(guac_rdpdr_device* devic
 
 static void guac_rdpdr_device_printer_free_handler(guac_rdpdr_device* device) {
 
-    guac_rdpdr_printer_data* printer_data =
-        (guac_rdpdr_printer_data*) device->data;
-
-    /* Close print job input (STDIN for filter process) if open */
-    if (printer_data->printer_input != -1)
-        close(printer_data->printer_input);
-
-    /* Close print job output (STDOUT for filter process) if open */
-    if (printer_data->printer_output != -1)
-        close(printer_data->printer_output);
-
-    /* Free underlying data */
-    free(device->data);
+    /* Close print job if open */
+    guac_rdpdr_print_job* job = (guac_rdpdr_print_job*) device->data;
+    if (job != NULL)
+        guac_rdpdr_print_job_free(job);
 
 }
 
@@ -469,7 +192,6 @@ void guac_rdpdr_register_printer(guac_rdpdrPlugin* rdpdr) {
 
     /* Get new device */
     guac_rdpdr_device* device = &(rdpdr->devices[id]);
-    guac_rdpdr_printer_data* printer_data;
 
     /* Init device */
     device->rdpdr       = rdpdr;
@@ -481,11 +203,8 @@ void guac_rdpdr_register_printer(guac_rdpdrPlugin* rdpdr) {
     device->iorequest_handler = guac_rdpdr_device_printer_iorequest_handler;
     device->free_handler      = guac_rdpdr_device_printer_free_handler;
 
-    /* Init data */
-    printer_data = malloc(sizeof(guac_rdpdr_printer_data));
-    printer_data->printer_input = -1;
-    printer_data->printer_output = -1;
-    device->data = printer_data;
+    /* No active print job yet */
+    device->data = NULL;
 
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-guacamole-server/blob/3fc43fba/src/protocols/rdp/guac_rdpdr/rdpdr_printer.h
----------------------------------------------------------------------
diff --git a/src/protocols/rdp/guac_rdpdr/rdpdr_printer.h 
b/src/protocols/rdp/guac_rdpdr/rdpdr_printer.h
index 72afa7a..6f203d6 100644
--- a/src/protocols/rdp/guac_rdpdr/rdpdr_printer.h
+++ b/src/protocols/rdp/guac_rdpdr/rdpdr_printer.h
@@ -32,41 +32,10 @@
 #endif
 
 /**
- * Data specific to an instance of the printer device.
- */
-typedef struct guac_rdpdr_printer_data {
-
-    /**
-     * File descriptor that should be written to when sending documents to the
-     * printer. If no print job is in progress, this will be -1.
-     */
-    int printer_input;
-
-    /**
-     * File descriptor that should be read from when receiving output from the
-     * printer. If no print job is in progress, this will be -1.
-     */
-    int printer_output;
-
-    /**
-     * The number of bytes received in the current print job.
-     */
-    int bytes_received;
-
-} guac_rdpdr_printer_data;
-
-/**
  * Registers a new printer device within the RDPDR plugin. This must be done
  * before RDPDR connection finishes.
  */
 void guac_rdpdr_register_printer(guac_rdpdrPlugin* rdpdr);
 
-/**
- * The command to run when filtering postscript to produce PDF. This must be
- * a NULL-terminated array of arguments, where the first argument is the name
- * of the file to run.
- */
-extern char* const guac_rdpdr_pdf_filter_command[];
-
 #endif
 


Reply via email to